From c081b21b1fe4fbad845088c4144da0bd2a8d89dc Mon Sep 17 00:00:00 2001 From: Alexander Ulanov Date: Mon, 2 Feb 2015 12:13:05 -0800 Subject: [PATCH 001/578] [MLLIB] SPARK-5491 (ex SPARK-1473): Chi-square feature selection The following is implemented: 1) generic traits for feature selection and filtering 2) trait for feature selection of LabeledPoint with discrete data 3) traits for calculation of contingency table and chi squared 4) class for chi-squared feature selection 5) tests for the above Needs some optimization in matrix operations. This request is a try to implement feature selection for MLLIB, the previous work by the issue author izendejas was not finished (https://issues.apache.org/jira/browse/SPARK-1473). This request is also related to data discretization issues: https://issues.apache.org/jira/browse/SPARK-1303 and https://issues.apache.org/jira/browse/SPARK-1216 that weren't merged. Author: Alexander Ulanov Closes #1484 from avulanov/featureselection and squashes the following commits: 755d358 [Alexander Ulanov] Addressing reviewers comments @mengxr a6ad82a [Alexander Ulanov] Addressing reviewers comments @mengxr 714b878 [Alexander Ulanov] Addressing reviewers comments @mengxr 010acff [Alexander Ulanov] Rebase 427ca4e [Alexander Ulanov] Addressing reviewers comments: implement VectorTransformer interface, use Statistics.chiSqTest f9b070a [Alexander Ulanov] Adding Apache header in tests... 80363ca [Alexander Ulanov] Tests, comments, apache headers and scala style 150a3e0 [Alexander Ulanov] Scala style fix f356365 [Alexander Ulanov] Chi Squared by contingency table. Refactoring 2bacdc7 [Alexander Ulanov] Combinations and chi-squared values test 66e0333 [Alexander Ulanov] Feature selector, fix of lazyness aab9b73 [Alexander Ulanov] Feature selection redesign with vigdorchik e24eee4 [Alexander Ulanov] Traits for FeatureSelection, CombinationsCalculator and FeatureFilter ca49e80 [Alexander Ulanov] Feature selection filter 2ade254 [Alexander Ulanov] Code style 0bd8434 [Alexander Ulanov] Chi Squared feature selection: initial version --- .../spark/mllib/feature/ChiSqSelector.scala | 127 ++++++++++++++++++ .../mllib/feature/ChiSqSelectorSuite.scala | 67 +++++++++ 2 files changed, 194 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala new file mode 100644 index 0000000000000..c6057c7f837b1 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.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.mllib.feature + +import scala.collection.mutable.ArrayBuilder + +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.Statistics +import org.apache.spark.rdd.RDD + +/** + * :: Experimental :: + * Chi Squared selector model. + * + * @param selectedFeatures list of indices to select (filter). Must be ordered asc + */ +@Experimental +class ChiSqSelectorModel (val selectedFeatures: Array[Int]) extends VectorTransformer { + + require(isSorted(selectedFeatures), "Array has to be sorted asc") + + protected def isSorted(array: Array[Int]): Boolean = { + var i = 1 + while (i < array.length) { + if (array(i) < array(i-1)) return false + i += 1 + } + true + } + + /** + * Applies transformation on a vector. + * + * @param vector vector to be transformed. + * @return transformed vector. + */ + override def transform(vector: Vector): Vector = { + compress(vector, selectedFeatures) + } + + /** + * Returns a vector with features filtered. + * Preserves the order of filtered features the same as their indices are stored. + * Might be moved to Vector as .slice + * @param features vector + * @param filterIndices indices of features to filter, must be ordered asc + */ + private def compress(features: Vector, filterIndices: Array[Int]): Vector = { + features match { + case SparseVector(size, indices, values) => + val newSize = filterIndices.length + val newValues = new ArrayBuilder.ofDouble + val newIndices = new ArrayBuilder.ofInt + var i = 0 + var j = 0 + var indicesIdx = 0 + var filterIndicesIdx = 0 + while (i < indices.length && j < filterIndices.length) { + indicesIdx = indices(i) + filterIndicesIdx = filterIndices(j) + if (indicesIdx == filterIndicesIdx) { + newIndices += j + newValues += values(i) + j += 1 + i += 1 + } else { + if (indicesIdx > filterIndicesIdx) { + j += 1 + } else { + i += 1 + } + } + } + // TODO: Sparse representation might be ineffective if (newSize ~= newValues.size) + Vectors.sparse(newSize, newIndices.result(), newValues.result()) + case DenseVector(values) => + val values = features.toArray + Vectors.dense(filterIndices.map(i => values(i))) + case other => + throw new UnsupportedOperationException( + s"Only sparse and dense vectors are supported but got ${other.getClass}.") + } + } +} + +/** + * :: Experimental :: + * Creates a ChiSquared feature selector. + * @param numTopFeatures number of features that selector will select + * (ordered by statistic value descending) + */ +@Experimental +class ChiSqSelector (val numTopFeatures: Int) { + + /** + * Returns a ChiSquared feature selector. + * + * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. + * Real-valued features will be treated as categorical for each distinct value. + * Apply feature discretizer before using this function. + */ + def fit(data: RDD[LabeledPoint]): ChiSqSelectorModel = { + val indices = Statistics.chiSqTest(data) + .zipWithIndex.sortBy { case (res, _) => -res.statistic } + .take(numTopFeatures) + .map { case (_, indices) => indices } + .sorted + new ChiSqSelectorModel(indices) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala new file mode 100644 index 0000000000000..747f5914598ec --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.feature + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.MLlibTestSparkContext + +class ChiSqSelectorSuite extends FunSuite with MLlibTestSparkContext { + + /* + * Contingency tables + * feature0 = {8.0, 0.0} + * class 0 1 2 + * 8.0||1|0|1| + * 0.0||0|2|0| + * + * feature1 = {7.0, 9.0} + * class 0 1 2 + * 7.0||1|0|0| + * 9.0||0|2|1| + * + * feature2 = {0.0, 6.0, 8.0, 5.0} + * class 0 1 2 + * 0.0||1|0|0| + * 6.0||0|1|0| + * 8.0||0|1|0| + * 5.0||0|0|1| + * + * Use chi-squared calculator from Internet + */ + + test("ChiSqSelector transform test (sparse & dense vector)") { + val labeledDiscreteData = sc.parallelize( + Seq(LabeledPoint(0.0, Vectors.sparse(3, Array((0, 8.0), (1, 7.0)))), + LabeledPoint(1.0, Vectors.sparse(3, Array((1, 9.0), (2, 6.0)))), + LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0, 8.0))), + LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0, 5.0)))), 2) + val preFilteredData = + Set(LabeledPoint(0.0, Vectors.dense(Array(0.0))), + LabeledPoint(1.0, Vectors.dense(Array(6.0))), + LabeledPoint(1.0, Vectors.dense(Array(8.0))), + LabeledPoint(2.0, Vectors.dense(Array(5.0)))) + val model = new ChiSqSelector(1).fit(labeledDiscreteData) + val filteredData = labeledDiscreteData.map { lp => + LabeledPoint(lp.label, model.transform(lp.features)) + }.collect().toSet + assert(filteredData == preFilteredData) + } +} From b2047b55c5fc85de6b63276d8ab9610d2496e08b Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 2 Feb 2015 12:27:08 -0800 Subject: [PATCH 002/578] SPARK-4585. Spark dynamic executor allocation should use minExecutors as... ... initial number Author: Sandy Ryza Closes #4051 from sryza/sandy-spark-4585 and squashes the following commits: d1dd039 [Sandy Ryza] Add spark.dynamicAllocation.initialNumExecutors and make min and max not required b7c59dc [Sandy Ryza] SPARK-4585. Spark dynamic executor allocation should use minExecutors as initial number --- .../spark/ExecutorAllocationManager.scala | 14 +++++++------ .../ExecutorAllocationManagerSuite.scala | 15 +++++++------- docs/configuration.md | 20 +++++++++++++------ docs/job-scheduling.md | 9 ++++----- .../spark/deploy/yarn/ClientArguments.scala | 17 ++++++++++++---- 5 files changed, 46 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index b28da192c1c0d..5d5288bb6e60d 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -49,6 +49,7 @@ import org.apache.spark.scheduler._ * spark.dynamicAllocation.enabled - Whether this feature is enabled * spark.dynamicAllocation.minExecutors - Lower bound on the number of executors * spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors + * spark.dynamicAllocation.initialExecutors - Number of executors to start with * * spark.dynamicAllocation.schedulerBacklogTimeout (M) - * If there are backlogged tasks for this duration, add new executors @@ -70,9 +71,10 @@ private[spark] class ExecutorAllocationManager( import ExecutorAllocationManager._ - // Lower and upper bounds on the number of executors. These are required. - private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1) - private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1) + // Lower and upper bounds on the number of executors. + private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", 0) + private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", + Integer.MAX_VALUE) // How long there must be backlogged tasks for before an addition is triggered private val schedulerBacklogTimeout = conf.getLong( @@ -132,10 +134,10 @@ private[spark] class ExecutorAllocationManager( */ private def validateSettings(): Unit = { if (minNumExecutors < 0 || maxNumExecutors < 0) { - throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!") + throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be positive!") } - if (minNumExecutors == 0 || maxNumExecutors == 0) { - throw new SparkException("spark.dynamicAllocation.{min/max}Executors cannot be 0!") + if (maxNumExecutors == 0) { + throw new SparkException("spark.dynamicAllocation.maxExecutors cannot be 0!") } if (minNumExecutors > maxNumExecutors) { throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " + diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 0e4df17c1bf87..57081ddd959a5 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -32,24 +32,23 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { import ExecutorAllocationManagerSuite._ test("verify min/max executors") { - // No min or max val conf = new SparkConf() .setMaster("local") .setAppName("test-executor-allocation-manager") .set("spark.dynamicAllocation.enabled", "true") .set("spark.dynamicAllocation.testing", "true") - intercept[SparkException] { new SparkContext(conf) } - SparkEnv.get.stop() // cleanup the created environment - SparkContext.clearActiveContext() + val sc0 = new SparkContext(conf) + assert(sc0.executorAllocationManager.isDefined) + sc0.stop() - // Only min - val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "1") + // Min < 0 + val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "-1") intercept[SparkException] { new SparkContext(conf1) } SparkEnv.get.stop() SparkContext.clearActiveContext() - // Only max - val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "2") + // Max < 0 + val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "-1") intercept[SparkException] { new SparkContext(conf2) } SparkEnv.get.stop() SparkContext.clearActiveContext() diff --git a/docs/configuration.md b/docs/configuration.md index e4e4b8d516b75..08c6befaf31ad 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1098,24 +1098,32 @@ Apart from these, the following properties are also available, and may be useful available on YARN mode. For more detail, see the description here.

- This requires the following configurations to be set: + This requires spark.shuffle.service.enabled to be set. + The following configurations are also relevant: spark.dynamicAllocation.minExecutors, spark.dynamicAllocation.maxExecutors, and - spark.shuffle.service.enabled + spark.dynamicAllocation.initialExecutors spark.dynamicAllocation.minExecutors - (none) + 0 - Lower bound for the number of executors if dynamic allocation is enabled (required). + Lower bound for the number of executors if dynamic allocation is enabled. spark.dynamicAllocation.maxExecutors - (none) + Integer.MAX_VALUE + + Upper bound for the number of executors if dynamic allocation is enabled. + + + + spark.dynamicAllocation.maxExecutors + spark.dynamicAllocation.minExecutors - Upper bound for the number of executors if dynamic allocation is enabled (required). + Initial number of executors to run if dynamic allocation is enabled. diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index a5425eb3557b2..5295e351dd711 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -77,11 +77,10 @@ scheduling while sharing cluster resources efficiently. ### Configuration and Setup All configurations used by this feature live under the `spark.dynamicAllocation.*` namespace. -To enable this feature, your application must set `spark.dynamicAllocation.enabled` to `true` and -provide lower and upper bounds for the number of executors through -`spark.dynamicAllocation.minExecutors` and `spark.dynamicAllocation.maxExecutors`. Other relevant -configurations are described on the [configurations page](configuration.html#dynamic-allocation) -and in the subsequent sections in detail. +To enable this feature, your application must set `spark.dynamicAllocation.enabled` to `true`. +Other relevant configurations are described on the +[configurations page](configuration.html#dynamic-allocation) and in the subsequent sections in +detail. Additionally, your application must use an external shuffle service. The purpose of the service is to preserve the shuffle files written by executors so the executors can be safely removed (more 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 f96b245512271..5eb2023802dfc 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 @@ -75,14 +75,23 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) .orElse(sparkConf.getOption("spark.yarn.dist.archives").map(p => Utils.resolveURIs(p))) .orElse(sys.env.get("SPARK_YARN_DIST_ARCHIVES")) .orNull - // If dynamic allocation is enabled, start at the max number of executors + // If dynamic allocation is enabled, start at the configured initial number of executors. + // Default to minExecutors if no initialExecutors is set. if (isDynamicAllocationEnabled) { + val minExecutorsConf = "spark.dynamicAllocation.minExecutors" + val initialExecutorsConf = "spark.dynamicAllocation.initialExecutors" val maxExecutorsConf = "spark.dynamicAllocation.maxExecutors" - if (!sparkConf.contains(maxExecutorsConf)) { + val minNumExecutors = sparkConf.getInt(minExecutorsConf, 0) + val initialNumExecutors = sparkConf.getInt(initialExecutorsConf, minNumExecutors) + val maxNumExecutors = sparkConf.getInt(maxExecutorsConf, Integer.MAX_VALUE) + + // If defined, initial executors must be between min and max + if (initialNumExecutors < minNumExecutors || initialNumExecutors > maxNumExecutors) { throw new IllegalArgumentException( - s"$maxExecutorsConf must be set if dynamic allocation is enabled!") + s"$initialExecutorsConf must be between $minExecutorsConf and $maxNumExecutors!") } - numExecutors = sparkConf.get(maxExecutorsConf).toInt + + numExecutors = initialNumExecutors } } From f5e63751f0ed50ceafdc2ec5173b161a5155b646 Mon Sep 17 00:00:00 2001 From: lianhuiwang Date: Mon, 2 Feb 2015 12:32:28 -0800 Subject: [PATCH 003/578] [SPARK-5173]support python application running on yarn cluster mode now when we run python application on yarn cluster mode through spark-submit, spark-submit does not support python application on yarn cluster mode. so i modify code of submit and yarn's AM in order to support it. through specifying .py file or primaryResource file via spark-submit, we can make pyspark run in yarn-cluster mode. example:spark-submit --master yarn-master --num-executors 1 --driver-memory 1g --executor-memory 1g xx.py --primaryResource yy.conf this config is same as pyspark on yarn-client mode. firstly,we put local path of .py or primaryResource to yarn's dist.files.that can be distributed on slave nodes.and then in spark-submit we transfer --py-files and --primaryResource to yarn.Client and use "org.apache.spark.deploy.PythonRunner" to user class that can run .py files on ApplicationMaster. in yarn.Client we transfer --py-files and --primaryResource to ApplicationMaster. in ApplicationMaster, user's class is org.apache.spark.deploy.PythonRunner, and user's args is primaryResource and -py-files. so that can make pyspark run on ApplicationMaster. JoshRosen tgravescs sryza Author: lianhuiwang Author: Wang Lianhui Closes #3976 from lianhuiwang/SPARK-5173 and squashes the following commits: 28a8a58 [lianhuiwang] fix variable name 67f8cee [lianhuiwang] update with andrewor's comments 0319ae3 [lianhuiwang] address with sryza's comments 2385ef6 [lianhuiwang] address with sryza's comments 03640ab [lianhuiwang] add sparkHome to env 47d2fc3 [lianhuiwang] fix test 2adc8f5 [lianhuiwang] add spark.test.home d60bc60 [lianhuiwang] fix test 5b30064 [lianhuiwang] add test 097a5ec [lianhuiwang] fix line length exceeds 100 905a106 [lianhuiwang] update with sryza and andrewor 's comments f1f55b6 [lianhuiwang] when yarn-cluster, all python files can be non-local 172eec1 [Wang Lianhui] fix a min submit's bug 9c941bc [lianhuiwang] support python application running on yarn cluster mode --- .../apache/spark/deploy/PythonRunner.scala | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 49 ++++++++++++++++--- .../spark/deploy/SparkSubmitArguments.scala | 12 ----- .../spark/deploy/yarn/ApplicationMaster.scala | 14 ++++-- .../yarn/ApplicationMasterArguments.scala | 13 +++++ .../org/apache/spark/deploy/yarn/Client.scala | 19 ++++++- .../spark/deploy/yarn/ClientArguments.scala | 20 ++++++-- .../spark/deploy/yarn/YarnClusterSuite.scala | 44 +++++++++++++++++ 8 files changed, 141 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala index 039c8719e2867..53e18c4bcec23 100644 --- a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -26,7 +26,7 @@ import org.apache.spark.api.python.PythonUtils import org.apache.spark.util.{RedirectThread, Utils} /** - * A main class used by spark-submit to launch Python applications. It executes python as a + * A main class used to launch Python applications. It executes python as a * subprocess and then has it connect back to the JVM to access system properties, etc. */ object PythonRunner { 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 c240bcd705d93..02021be9f93d4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -23,6 +23,8 @@ import java.net.URL import scala.collection.mutable.{ArrayBuffer, HashMap, Map} +import org.apache.hadoop.fs.Path + import org.apache.spark.executor.ExecutorURLClassLoader import org.apache.spark.util.Utils @@ -134,12 +136,27 @@ object SparkSubmit { } } + val isYarnCluster = clusterManager == YARN && deployMode == CLUSTER + + // 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) { + if (Utils.nonLocalPaths(args.primaryResource).nonEmpty) { + printErrorAndExit(s"Only local python files are supported: $args.primaryResource") + } + val nonLocalPyFiles = Utils.nonLocalPaths(args.pyFiles).mkString(",") + if (nonLocalPyFiles.nonEmpty) { + printErrorAndExit(s"Only local additional python files are supported: $nonLocalPyFiles") + } + } + // The following modes are not supported or applicable (clusterManager, deployMode) match { case (MESOS, CLUSTER) => printErrorAndExit("Cluster deploy mode is currently not supported for Mesos clusters.") - case (_, CLUSTER) if args.isPython => - printErrorAndExit("Cluster deploy mode is currently not supported for python applications.") + case (STANDALONE, CLUSTER) if args.isPython => + printErrorAndExit("Cluster deploy mode is currently not supported for python " + + "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) => @@ -150,7 +167,7 @@ object SparkSubmit { } // If we're running a python app, set the main class to our specific python runner - if (args.isPython) { + if (args.isPython && deployMode == CLIENT) { if (args.primaryResource == PYSPARK_SHELL) { args.mainClass = "py4j.GatewayServer" args.childArgs = ArrayBuffer("--die-on-broken-pipe", "0") @@ -167,6 +184,13 @@ 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) + } + // Special flag to avoid deprecation warnings at the client sysProps("SPARK_SUBMIT") = "true" @@ -245,7 +269,6 @@ 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 - val isYarnCluster = clusterManager == YARN && deployMode == CLUSTER if (!isYarnCluster && !args.isPython) { var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq.empty) if (isUserJar(args.primaryResource)) { @@ -270,10 +293,22 @@ object SparkSubmit { // In yarn-cluster mode, use yarn.Client as a wrapper around the user class if (isYarnCluster) { childMainClass = "org.apache.spark.deploy.yarn.Client" - if (args.primaryResource != SPARK_INTERNAL) { - childArgs += ("--jar", args.primaryResource) + if (args.isPython) { + val mainPyFile = new Path(args.primaryResource).getName + childArgs += ("--primary-py-file", mainPyFile) + if (args.pyFiles != null) { + // These files will be distributed to each machine's working directory, so strip the + // path prefix + val pyFilesNames = args.pyFiles.split(",").map(p => (new Path(p)).getName).mkString(",") + childArgs += ("--py-files", pyFilesNames) + } + childArgs += ("--class", "org.apache.spark.deploy.PythonRunner") + } else { + if (args.primaryResource != SPARK_INTERNAL) { + childArgs += ("--jar", args.primaryResource) + } + childArgs += ("--class", args.mainClass) } - childArgs += ("--class", args.mainClass) if (args.childArgs != null) { args.childArgs.foreach { arg => childArgs += ("--arg", arg) } } 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 81ec08cb6d501..73e921fd83ef2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -179,18 +179,6 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St SparkSubmit.printErrorAndExit("--py-files given but primary resource is not a Python script") } - // Require all python files to be local, so we can add them to the PYTHONPATH - if (isPython) { - if (Utils.nonLocalPaths(primaryResource).nonEmpty) { - SparkSubmit.printErrorAndExit(s"Only local python files are supported: $primaryResource") - } - val nonLocalPyFiles = Utils.nonLocalPaths(pyFiles).mkString(",") - if (nonLocalPyFiles.nonEmpty) { - SparkSubmit.printErrorAndExit( - s"Only local additional python files are supported: $nonLocalPyFiles") - } - } - if (master.startsWith("yarn")) { val hasHadoopEnv = env.contains("HADOOP_CONF_DIR") || env.contains("YARN_CONF_DIR") if (!hasHadoopEnv && !Utils.isTesting) { 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 d3e327b2497b7..eb328b2b8ac50 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 @@ -34,7 +34,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkEnv} import org.apache.spark.SparkException -import org.apache.spark.deploy.SparkHadoopUtil +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._ @@ -135,7 +135,7 @@ private[spark] class ApplicationMaster( .get().addShutdownHook(cleanupHook, ApplicationMaster.SHUTDOWN_HOOK_PRIORITY) // Call this to force generation of secret so it gets populated into the - // Hadoop UGI. This has to happen before the startUserClass which does a + // Hadoop UGI. This has to happen before the startUserApplication which does a // doAs in order for the credentials to be passed on to the executor containers. val securityMgr = new SecurityManager(sparkConf) @@ -254,7 +254,7 @@ private[spark] class ApplicationMaster( private def runDriver(securityMgr: SecurityManager): Unit = { addAmIpFilter() - userClassThread = startUserClass() + userClassThread = startUserApplication() // This a bit hacky, but we need to wait until the spark.driver.port property has // been set by the Thread executing the user class. @@ -448,9 +448,13 @@ private[spark] class ApplicationMaster( * * Returns the user thread that was started. */ - private def startUserClass(): Thread = { - logInfo("Starting the user JAR in a separate Thread") + private def startUserApplication(): Thread = { + logInfo("Starting the user application in a separate Thread") System.setProperty("spark.executor.instances", args.numExecutors.toString) + 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]]) 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 d76a63276d752..e1a992af3aae7 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 @@ -24,6 +24,8 @@ import collection.mutable.ArrayBuffer class ApplicationMasterArguments(val args: Array[String]) { var userJar: String = null var userClass: String = null + var primaryPyFile: String = null + var pyFiles: String = null var userArgs: Seq[String] = Seq[String]() var executorMemory = 1024 var executorCores = 1 @@ -48,6 +50,14 @@ class ApplicationMasterArguments(val args: Array[String]) { userClass = value args = tail + case ("--primary-py-file") :: value :: tail => + primaryPyFile = value + args = tail + + case ("--py-files") :: value :: tail => + pyFiles = value + args = tail + case ("--args" | "--arg") :: value :: tail => userArgsBuffer += value args = tail @@ -81,6 +91,9 @@ class ApplicationMasterArguments(val args: Array[String]) { |Options: | --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 + | --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. | Multiple invocations are possible, each will be passed in order. | --num-executors NUM Number of executors to start (Default: 2) 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 1a18e6509ef26..91e8574e94e2f 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 @@ -21,7 +21,7 @@ import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} import java.nio.ByteBuffer import scala.collection.JavaConversions._ -import scala.collection.mutable.{HashMap, ListBuffer, Map} +import scala.collection.mutable.{ArrayBuffer, HashMap, ListBuffer, Map} import scala.util.{Try, Success, Failure} import com.google.common.base.Objects @@ -477,17 +477,32 @@ private[spark] class Client( } else { Nil } + val primaryPyFile = + if (args.primaryPyFile != null) { + Seq("--primary-py-file", args.primaryPyFile) + } else { + Nil + } + val pyFiles = + if (args.pyFiles != null) { + Seq("--py-files", args.pyFiles) + } else { + Nil + } val amClass = if (isClusterMode) { Class.forName("org.apache.spark.deploy.yarn.ApplicationMaster").getName } else { Class.forName("org.apache.spark.deploy.yarn.ExecutorLauncher").getName } + if (args.primaryPyFile != null && args.primaryPyFile.endsWith(".py")) { + args.userArgs = ArrayBuffer(args.primaryPyFile, args.pyFiles) ++ args.userArgs + } val userArgs = args.userArgs.flatMap { arg => Seq("--arg", YarnSparkHadoopUtil.escapeForShell(arg)) } val amArgs = - Seq(amClass) ++ userClass ++ userJar ++ userArgs ++ + Seq(amClass) ++ userClass ++ userJar ++ primaryPyFile ++ pyFiles ++ userArgs ++ Seq( "--executor-memory", args.executorMemory.toString + "m", "--executor-cores", args.executorCores.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 5eb2023802dfc..3bc7eb1abf341 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 @@ -30,7 +30,9 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) var archives: String = null var userJar: String = null var userClass: String = null - var userArgs: Seq[String] = Seq[String]() + var pyFiles: String = null + var primaryPyFile: String = null + var userArgs: ArrayBuffer[String] = new ArrayBuffer[String]() var executorMemory = 1024 // MB var executorCores = 1 var numExecutors = DEFAULT_NUMBER_EXECUTORS @@ -132,7 +134,6 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) } private def parseArgs(inputArgs: List[String]): Unit = { - val userArgsBuffer = new ArrayBuffer[String]() var args = inputArgs while (!args.isEmpty) { @@ -145,11 +146,15 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) userClass = value args = tail + case ("--primary-py-file") :: value :: tail => + primaryPyFile = value + args = tail + case ("--args" | "--arg") :: value :: tail => if (args(0) == "--args") { println("--args is deprecated. Use --arg instead.") } - userArgsBuffer += value + userArgs += value args = tail case ("--master-class" | "--am-class") :: value :: tail => @@ -205,6 +210,10 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) addJars = value args = tail + case ("--py-files") :: value :: tail => + pyFiles = value + args = tail + case ("--files") :: value :: tail => files = value args = tail @@ -219,8 +228,6 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) throw new IllegalArgumentException(getUsageMessage(args)) } } - - userArgs = userArgsBuffer.readOnly } private def getUsageMessage(unknownParam: List[String] = null): String = { @@ -232,6 +239,7 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) | --jar JAR_PATH Path to your application's JAR file (required in yarn-cluster | mode) | --class CLASS_NAME Name of your application's main class (required) + | --primary-py-file A main Python 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) @@ -244,6 +252,8 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) | 'default') | --addJars jars Comma separated list of local jars that want SparkContext.addJar | to work with. + | --py-files PY_FILES Comma-separated list of .zip, .egg, or .py files to + | place on the PYTHONPATH for Python apps. | --files files Comma separated list of files to be distributed with the job. | --archives archives Comma separated list of archives to be distributed with the job. """.stripMargin 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 d79b85e867fcd..7165918e1bfcf 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 @@ -45,6 +45,29 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n """.stripMargin + private val TEST_PYFILE = """ + |import sys + |from operator import add + | + |from pyspark import SparkConf , SparkContext + |if __name__ == "__main__": + | if len(sys.argv) != 3: + | print >> sys.stderr, "Usage: test.py [master] [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') + | result = "failure" + | rdd = sc.parallelize(range(10)) + | cnt = rdd.count() + | if cnt == 10: + | result = "success" + | status.write(result) + | status.close() + | sc.stop() + """.stripMargin + private var yarnCluster: MiniYARNCluster = _ private var tempDir: File = _ private var fakeSparkJar: File = _ @@ -98,6 +121,9 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit } 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) @@ -146,6 +172,24 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit 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) + val pyFile = new File(tempDir, "test2.py") + Files.write(TEST_PYFILE, pyFile, Charsets.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) + checkResult(result) + } + /** * 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 From 3f941b68a2336aa7876aeda99865e7c19b53bc5c Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Mon, 2 Feb 2015 12:33:49 -0800 Subject: [PATCH 004/578] [Docs] Fix Building Spark link text Author: Nicholas Chammas Closes #4312 from nchammas/patch-2 and squashes the following commits: 9d943aa [Nicholas Chammas] [Docs] Fix Building Spark link text --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 16628bd406775..af02339578195 100644 --- a/README.md +++ b/README.md @@ -26,7 +26,7 @@ To build Spark and its example programs, run: (You do not need to do this if you downloaded a pre-built package.) More detailed documentation is available from the project site, at -["Building Spark with Maven"](http://spark.apache.org/docs/latest/building-spark.html). +["Building Spark"](http://spark.apache.org/docs/latest/building-spark.html). ## Interactive Scala Shell From 62a93a1698e8d0a667eb1718ba75dcfc86eabaaf Mon Sep 17 00:00:00 2001 From: Xutingjun <1039320815@qq.com> Date: Mon, 2 Feb 2015 12:37:51 -0800 Subject: [PATCH 005/578] [SPARK-5530] Add executor container to executorIdToContainer when call killExecutor method, it will only go to the else branch, because the variable executorIdToContainer never be put any value. Author: Xutingjun <1039320815@qq.com> Closes #4309 from XuTingjun/dynamicAllocator and squashes the following commits: c823418 [Xutingjun] fix bugwq --- .../main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala | 1 + 1 file changed, 1 insertion(+) 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 3849586c6111e..040406c15030e 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 @@ -300,6 +300,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 val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, new HashSet[ContainerId]) From 683e938242e29a0d584452e5230b4168b85bdab2 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 2 Feb 2015 13:53:55 -0800 Subject: [PATCH 006/578] [SPARK-5212][SQL] Add support of schema-less, custom field delimiter and SerDe for HiveQL transform This pr adds the support of schema-less syntax, custom field delimiter and SerDe for HiveQL's transform. Author: Liang-Chi Hsieh Closes #4014 from viirya/schema_less_trans and squashes the following commits: ac2d1fe [Liang-Chi Hsieh] Refactor codes for comments. a137933 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into schema_less_trans aa10fbd [Liang-Chi Hsieh] Add Hive golden answer files again. 575f695 [Liang-Chi Hsieh] Add Hive golden answer files for new unit tests. a422562 [Liang-Chi Hsieh] Use createQueryTest for unit tests and remove unnecessary imports. ccb71e3 [Liang-Chi Hsieh] Refactor codes for comments. 37bd391 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into schema_less_trans 6000889 [Liang-Chi Hsieh] Wrap input and output schema into ScriptInputOutputSchema. 21727f7 [Liang-Chi Hsieh] Move schema-less output to proper place. Use multilines instead of a long line SQL. 9a6dc04 [Liang-Chi Hsieh] setRecordReaderID is introduced in 0.13.1, use reflection API to call it. 7a14f31 [Liang-Chi Hsieh] Fix bug. 799b5e1 [Liang-Chi Hsieh] Call getSerializedClass instead of using Text. be2c3fc [Liang-Chi Hsieh] Fix style. 32d3046 [Liang-Chi Hsieh] Add SerDe support. ab22f7b [Liang-Chi Hsieh] Fix style. 7a48e42 [Liang-Chi Hsieh] Add support of custom field delimiter. b1729d9 [Liang-Chi Hsieh] Fix style. ccee49e [Liang-Chi Hsieh] Add unit test. f561c37 [Liang-Chi Hsieh] Add support of schema-less script transformation. --- .../plans/logical/ScriptTransformation.scala | 10 +- .../org/apache/spark/sql/hive/HiveQl.scala | 59 ++- .../spark/sql/hive/HiveStrategies.scala | 4 +- .../hive/execution/ScriptTransformation.scala | 220 +++++++- ...ansform-0-d5738de14dd6e29da712ec3318f4118f | 500 ++++++++++++++++++ ...ansform-1-49624ef4e2c3cc2040c06660b926219b | 500 ++++++++++++++++++ ...h SerDe-0-cdc393f3914c879787efe523f692b1e0 | 500 ++++++++++++++++++ ... SerDe3-0-58a8b7eb07a949bc44dccb723222957f | 500 ++++++++++++++++++ ... SerDe4-0-ba9ad2499a7408cb350c7abafaf9ea97 | 500 ++++++++++++++++++ ...limiter-0-703cca3c02ced422feb11dc13b744484 | 500 ++++++++++++++++++ ...limiter-0-82639dda9ba42df817466dffe2929174 | 500 ++++++++++++++++++ ...imiter2-0-e8713b21483e1efb78ee90b61530479b | 500 ++++++++++++++++++ ...imiter2-0-e8d2b2e60551f69bfb44e555f5cff064 | 500 ++++++++++++++++++ ...imiter3-0-d4f4f471819345e9ce1964e281ea5289 | 500 ++++++++++++++++++ .../sql/hive/execution/HiveQuerySuite.scala | 76 ++- .../org/apache/spark/sql/hive/Shim12.scala | 6 +- .../org/apache/spark/sql/hive/Shim13.scala | 19 +- 17 files changed, 5360 insertions(+), 34 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/schema-less transform-0-d5738de14dd6e29da712ec3318f4118f create mode 100644 sql/hive/src/test/resources/golden/schema-less transform-1-49624ef4e2c3cc2040c06660b926219b create mode 100644 sql/hive/src/test/resources/golden/transform with SerDe-0-cdc393f3914c879787efe523f692b1e0 create mode 100644 sql/hive/src/test/resources/golden/transform with SerDe3-0-58a8b7eb07a949bc44dccb723222957f create mode 100644 sql/hive/src/test/resources/golden/transform with SerDe4-0-ba9ad2499a7408cb350c7abafaf9ea97 create mode 100644 sql/hive/src/test/resources/golden/transform with custom field delimiter-0-703cca3c02ced422feb11dc13b744484 create mode 100644 sql/hive/src/test/resources/golden/transform with custom field delimiter-0-82639dda9ba42df817466dffe2929174 create mode 100644 sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8713b21483e1efb78ee90b61530479b create mode 100644 sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8d2b2e60551f69bfb44e555f5cff064 create mode 100644 sql/hive/src/test/resources/golden/transform with custom field delimiter3-0-d4f4f471819345e9ce1964e281ea5289 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 4460c86ed9026..cfe2c7a39a17c 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 @@ -25,9 +25,17 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} * @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. + * @param ioschema the input and output schema applied in the execution of the script. */ case class ScriptTransformation( input: Seq[Expression], script: String, output: Seq[Attribute], - child: LogicalPlan) extends UnaryNode + child: LogicalPlan, + ioschema: ScriptInputOutputSchema) extends UnaryNode + +/** + * A placeholder for implementation specific input and output properties when passing data + * to a script. For example, in Hive this would specify which SerDes to use. + */ +trait ScriptInputOutputSchema 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 30a64b48d7951..ab305e1f82a55 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 @@ -33,7 +33,7 @@ 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.execution.ExplainCommand -import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable} +import org.apache.spark.sql.hive.execution.{HiveNativeCommand, DropTable, AnalyzeTable, HiveScriptIOSchema} import org.apache.spark.sql.types._ /* Implicit conversions */ @@ -627,29 +627,64 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token("TOK_SELEXPR", Token("TOK_TRANSFORM", Token("TOK_EXPLIST", inputExprs) :: - Token("TOK_SERDE", Nil) :: + Token("TOK_SERDE", inputSerdeClause) :: Token("TOK_RECORDWRITER", writerClause) :: // TODO: Need to support other types of (in/out)put Token(script, Nil) :: - Token("TOK_SERDE", serdeClause) :: + Token("TOK_SERDE", outputSerdeClause) :: Token("TOK_RECORDREADER", readerClause) :: - outputClause :: Nil) :: Nil) => - - val output = outputClause match { - case Token("TOK_ALIASLIST", aliases) => - aliases.map { case Token(name, Nil) => AttributeReference(name, StringType)() } - case Token("TOK_TABCOLLIST", attributes) => - attributes.map { case Token("TOK_TABCOL", Token(name, Nil) :: dataType :: Nil) => - AttributeReference(name, nodeToDataType(dataType))() } + outputClause) :: Nil) => + + val (output, schemaLess) = outputClause match { + case Token("TOK_ALIASLIST", aliases) :: Nil => + (aliases.map { case Token(name, Nil) => AttributeReference(name, StringType)() }, + false) + case Token("TOK_TABCOLLIST", attributes) :: Nil => + (attributes.map { case Token("TOK_TABCOL", Token(name, Nil) :: dataType :: Nil) => + AttributeReference(name, nodeToDataType(dataType))() }, false) + case Nil => + (List(AttributeReference("key", StringType)(), + AttributeReference("value", StringType)()), true) } + + def matchSerDe(clause: Seq[ASTNode]) = clause match { + case Token("TOK_SERDEPROPS", propsClause) :: Nil => + val rowFormat = propsClause.map { + case Token(name, Token(value, Nil) :: Nil) => (name, value) + } + (rowFormat, "", Nil) + + case Token("TOK_SERDENAME", Token(serdeClass, Nil) :: Nil) :: Nil => + (Nil, serdeClass, Nil) + + case Token("TOK_SERDENAME", Token(serdeClass, Nil) :: + Token("TOK_TABLEPROPERTIES", + Token("TOK_TABLEPROPLIST", propsClause) :: Nil) :: Nil) :: Nil => + val serdeProps = propsClause.map { + case Token("TOK_TABLEPROPERTY", Token(name, Nil) :: Token(value, Nil) :: Nil) => + (name, value) + } + (Nil, serdeClass, serdeProps) + + case Nil => (Nil, "", Nil) + } + + val (inRowFormat, inSerdeClass, inSerdeProps) = matchSerDe(inputSerdeClause) + val (outRowFormat, outSerdeClass, outSerdeProps) = matchSerDe(outputSerdeClause) + val unescapedScript = BaseSemanticAnalyzer.unescapeSQLString(script) + val schema = HiveScriptIOSchema( + inRowFormat, outRowFormat, + inSerdeClass, outSerdeClass, + inSerdeProps, outSerdeProps, schemaLess) + Some( logical.ScriptTransformation( inputExprs.map(nodeToExpr), unescapedScript, output, - withWhere)) + withWhere, schema)) case _ => None } 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 ace9329cd5821..83244ce1e372a 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 @@ -166,8 +166,8 @@ private[hive] trait HiveStrategies { object Scripts extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.ScriptTransformation(input, script, output, child) => - ScriptTransformation(input, script, output, planLater(child))(hiveContext) :: Nil + case logical.ScriptTransformation(input, script, output, child, schema: HiveScriptIOSchema) => + ScriptTransformation(input, script, output, planLater(child), schema)(hiveContext) :: Nil case _ => Nil } } 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 0c8f676e9c5c8..c54fbb6e24690 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 @@ -18,11 +18,26 @@ package org.apache.spark.sql.hive.execution import java.io.{BufferedReader, InputStreamReader} +import java.io.{DataInputStream, DataOutputStream, EOFException} +import java.util.Properties + +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._ -import org.apache.spark.sql.hive.HiveContext +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.util.Utils + /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -40,7 +55,8 @@ case class ScriptTransformation( input: Seq[Expression], script: String, output: Seq[Attribute], - child: SparkPlan)(@transient sc: HiveContext) + child: SparkPlan, + ioschema: HiveScriptIOSchema)(@transient sc: HiveContext) extends UnaryNode { override def otherCopyArgs = sc :: Nil @@ -53,28 +69,202 @@ case class ScriptTransformation( val inputStream = proc.getInputStream val outputStream = proc.getOutputStream val reader = new BufferedReader(new InputStreamReader(inputStream)) + + val (outputSerde, outputSoi) = ioschema.initOutputSerDe(output) + + val iterator: Iterator[Row] = new Iterator[Row] with HiveInspectors { + var cacheRow: Row = null + var curLine: String = null + var eof: Boolean = false + + override def hasNext: Boolean = { + if (outputSerde == null) { + if (curLine == null) { + curLine = reader.readLine() + curLine != null + } else { + true + } + } else { + !eof + } + } + + def deserialize(): Row = { + if (cacheRow != null) return cacheRow + + val mutableRow = new SpecificMutableRow(output.map(_.dataType)) + try { + val dataInputStream = new DataInputStream(inputStream) + val writable = outputSerde.getSerializedClass().newInstance + writable.readFields(dataInputStream) + + val raw = outputSerde.deserialize(writable) + val dataList = outputSoi.getStructFieldsDataAsList(raw) + val fieldList = outputSoi.getAllStructFieldRefs() + + var i = 0 + dataList.foreach( element => { + if (element == null) { + mutableRow.setNullAt(i) + } else { + mutableRow(i) = unwrap(element, fieldList(i).getFieldObjectInspector) + } + i += 1 + }) + return mutableRow + } catch { + case e: EOFException => + eof = true + return null + } + } - // TODO: This should be exposed as an iterator instead of reading in all the data at once. - val outputLines = collection.mutable.ArrayBuffer[Row]() - val readerThread = new Thread("Transform OutputReader") { - override def run() { - var curLine = reader.readLine() - while (curLine != null) { - // TODO: Use SerDe - outputLines += new GenericRow(curLine.split("\t").asInstanceOf[Array[Any]]) + override def next(): Row = { + if (!hasNext) { + throw new NoSuchElementException + } + + if (outputSerde == null) { + val prevLine = curLine curLine = reader.readLine() + + if (!ioschema.schemaLess) { + new GenericRow( + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD")) + .asInstanceOf[Array[Any]]) + } else { + new GenericRow( + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2) + .asInstanceOf[Array[Any]]) + } + } else { + val ret = deserialize() + if (!eof) { + cacheRow = null + cacheRow = deserialize() + } + ret } } } - readerThread.start() + + val (inputSerde, inputSoi) = ioschema.initInputSerDe(input) + val dataOutputStream = new DataOutputStream(outputStream) val outputProjection = new InterpretedProjection(input, child.output) + iter .map(outputProjection) - // TODO: Use SerDe - .map(_.mkString("", "\t", "\n").getBytes("utf-8")).foreach(outputStream.write) + .foreach { row => + if (inputSerde == null) { + val data = row.mkString("", ioschema.inputRowFormatMap("TOK_TABLEROWFORMATFIELD"), + ioschema.inputRowFormatMap("TOK_TABLEROWFORMATLINES")).getBytes("utf-8") + + outputStream.write(data) + } else { + val writable = inputSerde.serialize(row.asInstanceOf[GenericRow].values, inputSoi) + prepareWritable(writable).write(dataOutputStream) + } + } outputStream.close() - readerThread.join() - outputLines.toIterator + iterator + } + } +} + +/** + * The wrapper class of Hive input and output schema properties + */ +case class HiveScriptIOSchema ( + inputRowFormat: Seq[(String, String)], + outputRowFormat: Seq[(String, String)], + inputSerdeClass: String, + outputSerdeClass: String, + inputSerdeProps: Seq[(String, String)], + outputSerdeProps: Seq[(String, String)], + schemaLess: Boolean) extends ScriptInputOutputSchema with HiveInspectors { + + val defaultFormat = Map(("TOK_TABLEROWFORMATFIELD", "\t"), + ("TOK_TABLEROWFORMATLINES", "\n")) + + val inputRowFormatMap = inputRowFormat.toMap.withDefault((k) => defaultFormat(k)) + val outputRowFormatMap = outputRowFormat.toMap.withDefault((k) => defaultFormat(k)) + + + def initInputSerDe(input: Seq[Expression]): (AbstractSerDe, ObjectInspector) = { + val (columns, columnTypes) = parseAttrs(input) + val serde = initSerDe(inputSerdeClass, columns, columnTypes, inputSerdeProps) + (serde, initInputSoi(serde, columns, columnTypes)) + } + + def initOutputSerDe(output: Seq[Attribute]): (AbstractSerDe, StructObjectInspector) = { + val (columns, columnTypes) = parseAttrs(output) + val serde = initSerDe(outputSerdeClass, columns, columnTypes, outputSerdeProps) + (serde, initOutputputSoi(serde)) + } + + def parseAttrs(attrs: Seq[Expression]): (Seq[String], Seq[DataType]) = { + + val columns = attrs.map { + case aref: AttributeReference => aref.name + case e: NamedExpression => e.name + case _ => null + } + + val columnTypes = attrs.map { + case aref: AttributeReference => aref.dataType + case e: NamedExpression => e.dataType + case _ => null + } + + (columns, columnTypes) + } + + def initSerDe(serdeClassName: String, columns: Seq[String], + columnTypes: Seq[DataType], serdeProps: Seq[(String, String)]): AbstractSerDe = { + + val serde: AbstractSerDe = if (serdeClassName != "") { + val trimed_class = serdeClassName.split("'")(1) + Utils.classForName(trimed_class) + .newInstance.asInstanceOf[AbstractSerDe] + } else { + null + } + + if (serde != null) { + val columnTypesNames = columnTypes.map(_.toTypeInfo.getTypeName()).mkString(",") + + var propsMap = serdeProps.map(kv => { + (kv._1.split("'")(1), kv._2.split("'")(1)) + }).toMap + (serdeConstants.LIST_COLUMNS -> columns.mkString(",")) + propsMap = propsMap + (serdeConstants.LIST_COLUMN_TYPES -> columnTypesNames) + + val properties = new Properties() + properties.putAll(propsMap) + serde.initialize(null, properties) + } + + serde + } + + def initInputSoi(inputSerde: AbstractSerDe, columns: Seq[String], columnTypes: Seq[DataType]) + : ObjectInspector = { + + if (inputSerde != null) { + val fieldObjectInspectors = columnTypes.map(toInspector(_)) + ObjectInspectorFactory + .getStandardStructObjectInspector(columns, fieldObjectInspectors) + .asInstanceOf[ObjectInspector] + } else { + null + } + } + + def initOutputputSoi(outputSerde: AbstractSerDe): StructObjectInspector = { + if (outputSerde != null) { + outputSerde.getObjectInspector().asInstanceOf[StructObjectInspector] + } else { + null } } } diff --git a/sql/hive/src/test/resources/golden/schema-less transform-0-d5738de14dd6e29da712ec3318f4118f b/sql/hive/src/test/resources/golden/schema-less transform-0-d5738de14dd6e29da712ec3318f4118f new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/golden/schema-less transform-0-d5738de14dd6e29da712ec3318f4118f @@ -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/schema-less transform-1-49624ef4e2c3cc2040c06660b926219b b/sql/hive/src/test/resources/golden/schema-less transform-1-49624ef4e2c3cc2040c06660b926219b new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/golden/schema-less transform-1-49624ef4e2c3cc2040c06660b926219b @@ -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/transform with SerDe-0-cdc393f3914c879787efe523f692b1e0 b/sql/hive/src/test/resources/golden/transform with SerDe-0-cdc393f3914c879787efe523f692b1e0 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with SerDe-0-cdc393f3914c879787efe523f692b1e0 @@ -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/transform with SerDe3-0-58a8b7eb07a949bc44dccb723222957f b/sql/hive/src/test/resources/golden/transform with SerDe3-0-58a8b7eb07a949bc44dccb723222957f new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with SerDe3-0-58a8b7eb07a949bc44dccb723222957f @@ -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/transform with SerDe4-0-ba9ad2499a7408cb350c7abafaf9ea97 b/sql/hive/src/test/resources/golden/transform with SerDe4-0-ba9ad2499a7408cb350c7abafaf9ea97 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with SerDe4-0-ba9ad2499a7408cb350c7abafaf9ea97 @@ -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/transform with custom field delimiter-0-703cca3c02ced422feb11dc13b744484 b/sql/hive/src/test/resources/golden/transform with custom field delimiter-0-703cca3c02ced422feb11dc13b744484 new file mode 100644 index 0000000000000..e34118512c1d7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with custom field delimiter-0-703cca3c02ced422feb11dc13b744484 @@ -0,0 +1,500 @@ +238 +86 +311 +27 +165 +409 +255 +278 +98 +484 +265 +193 +401 +150 +273 +224 +369 +66 +128 +213 +146 +406 +429 +374 +152 +469 +145 +495 +37 +327 +281 +277 +209 +15 +82 +403 +166 +417 +430 +252 +292 +219 +287 +153 +193 +338 +446 +459 +394 +237 +482 +174 +413 +494 +207 +199 +466 +208 +174 +399 +396 +247 +417 +489 +162 +377 +397 +309 +365 +266 +439 +342 +367 +325 +167 +195 +475 +17 +113 +155 +203 +339 +0 +455 +128 +311 +316 +57 +302 +205 +149 +438 +345 +129 +170 +20 +489 +157 +378 +221 +92 +111 +47 +72 +4 +280 +35 +427 +277 +208 +356 +399 +169 +382 +498 +125 +386 +437 +469 +192 +286 +187 +176 +54 +459 +51 +138 +103 +239 +213 +216 +430 +278 +176 +289 +221 +65 +318 +332 +311 +275 +137 +241 +83 +333 +180 +284 +12 +230 +181 +67 +260 +404 +384 +489 +353 +373 +272 +138 +217 +84 +348 +466 +58 +8 +411 +230 +208 +348 +24 +463 +431 +179 +172 +42 +129 +158 +119 +496 +0 +322 +197 +468 +393 +454 +100 +298 +199 +191 +418 +96 +26 +165 +327 +230 +205 +120 +131 +51 +404 +43 +436 +156 +469 +468 +308 +95 +196 +288 +481 +457 +98 +282 +197 +187 +318 +318 +409 +470 +137 +369 +316 +169 +413 +85 +77 +0 +490 +87 +364 +179 +118 +134 +395 +282 +138 +238 +419 +15 +118 +72 +90 +307 +19 +435 +10 +277 +273 +306 +224 +309 +389 +327 +242 +369 +392 +272 +331 +401 +242 +452 +177 +226 +5 +497 +402 +396 +317 +395 +58 +35 +336 +95 +11 +168 +34 +229 +233 +143 +472 +322 +498 +160 +195 +42 +321 +430 +119 +489 +458 +78 +76 +41 +223 +492 +149 +449 +218 +228 +138 +453 +30 +209 +64 +468 +76 +74 +342 +69 +230 +33 +368 +103 +296 +113 +216 +367 +344 +167 +274 +219 +239 +485 +116 +223 +256 +263 +70 +487 +480 +401 +288 +191 +5 +244 +438 +128 +467 +432 +202 +316 +229 +469 +463 +280 +2 +35 +283 +331 +235 +80 +44 +193 +321 +335 +104 +466 +366 +175 +403 +483 +53 +105 +257 +406 +409 +190 +406 +401 +114 +258 +90 +203 +262 +348 +424 +12 +396 +201 +217 +164 +431 +454 +478 +298 +125 +431 +164 +424 +187 +382 +5 +70 +397 +480 +291 +24 +351 +255 +104 +70 +163 +438 +119 +414 +200 +491 +237 +439 +360 +248 +479 +305 +417 +199 +444 +120 +429 +169 +443 +323 +325 +277 +230 +478 +178 +468 +310 +317 +333 +493 +460 +207 +249 +265 +480 +83 +136 +353 +172 +214 +462 +233 +406 +133 +175 +189 +454 +375 +401 +421 +407 +384 +256 +26 +134 +67 +384 +379 +18 +462 +492 +100 +298 +9 +341 +498 +146 +458 +362 +186 +285 +348 +167 +18 +273 +183 +281 +344 +97 +469 +315 +84 +28 +37 +448 +152 +348 +307 +194 +414 +477 +222 +126 +90 +169 +403 +400 +200 +97 diff --git a/sql/hive/src/test/resources/golden/transform with custom field delimiter-0-82639dda9ba42df817466dffe2929174 b/sql/hive/src/test/resources/golden/transform with custom field delimiter-0-82639dda9ba42df817466dffe2929174 new file mode 100644 index 0000000000000..e34118512c1d7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with custom field delimiter-0-82639dda9ba42df817466dffe2929174 @@ -0,0 +1,500 @@ +238 +86 +311 +27 +165 +409 +255 +278 +98 +484 +265 +193 +401 +150 +273 +224 +369 +66 +128 +213 +146 +406 +429 +374 +152 +469 +145 +495 +37 +327 +281 +277 +209 +15 +82 +403 +166 +417 +430 +252 +292 +219 +287 +153 +193 +338 +446 +459 +394 +237 +482 +174 +413 +494 +207 +199 +466 +208 +174 +399 +396 +247 +417 +489 +162 +377 +397 +309 +365 +266 +439 +342 +367 +325 +167 +195 +475 +17 +113 +155 +203 +339 +0 +455 +128 +311 +316 +57 +302 +205 +149 +438 +345 +129 +170 +20 +489 +157 +378 +221 +92 +111 +47 +72 +4 +280 +35 +427 +277 +208 +356 +399 +169 +382 +498 +125 +386 +437 +469 +192 +286 +187 +176 +54 +459 +51 +138 +103 +239 +213 +216 +430 +278 +176 +289 +221 +65 +318 +332 +311 +275 +137 +241 +83 +333 +180 +284 +12 +230 +181 +67 +260 +404 +384 +489 +353 +373 +272 +138 +217 +84 +348 +466 +58 +8 +411 +230 +208 +348 +24 +463 +431 +179 +172 +42 +129 +158 +119 +496 +0 +322 +197 +468 +393 +454 +100 +298 +199 +191 +418 +96 +26 +165 +327 +230 +205 +120 +131 +51 +404 +43 +436 +156 +469 +468 +308 +95 +196 +288 +481 +457 +98 +282 +197 +187 +318 +318 +409 +470 +137 +369 +316 +169 +413 +85 +77 +0 +490 +87 +364 +179 +118 +134 +395 +282 +138 +238 +419 +15 +118 +72 +90 +307 +19 +435 +10 +277 +273 +306 +224 +309 +389 +327 +242 +369 +392 +272 +331 +401 +242 +452 +177 +226 +5 +497 +402 +396 +317 +395 +58 +35 +336 +95 +11 +168 +34 +229 +233 +143 +472 +322 +498 +160 +195 +42 +321 +430 +119 +489 +458 +78 +76 +41 +223 +492 +149 +449 +218 +228 +138 +453 +30 +209 +64 +468 +76 +74 +342 +69 +230 +33 +368 +103 +296 +113 +216 +367 +344 +167 +274 +219 +239 +485 +116 +223 +256 +263 +70 +487 +480 +401 +288 +191 +5 +244 +438 +128 +467 +432 +202 +316 +229 +469 +463 +280 +2 +35 +283 +331 +235 +80 +44 +193 +321 +335 +104 +466 +366 +175 +403 +483 +53 +105 +257 +406 +409 +190 +406 +401 +114 +258 +90 +203 +262 +348 +424 +12 +396 +201 +217 +164 +431 +454 +478 +298 +125 +431 +164 +424 +187 +382 +5 +70 +397 +480 +291 +24 +351 +255 +104 +70 +163 +438 +119 +414 +200 +491 +237 +439 +360 +248 +479 +305 +417 +199 +444 +120 +429 +169 +443 +323 +325 +277 +230 +478 +178 +468 +310 +317 +333 +493 +460 +207 +249 +265 +480 +83 +136 +353 +172 +214 +462 +233 +406 +133 +175 +189 +454 +375 +401 +421 +407 +384 +256 +26 +134 +67 +384 +379 +18 +462 +492 +100 +298 +9 +341 +498 +146 +458 +362 +186 +285 +348 +167 +18 +273 +183 +281 +344 +97 +469 +315 +84 +28 +37 +448 +152 +348 +307 +194 +414 +477 +222 +126 +90 +169 +403 +400 +200 +97 diff --git a/sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8713b21483e1efb78ee90b61530479b b/sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8713b21483e1efb78ee90b61530479b new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8713b21483e1efb78ee90b61530479b @@ -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/transform with custom field delimiter2-0-e8d2b2e60551f69bfb44e555f5cff064 b/sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8d2b2e60551f69bfb44e555f5cff064 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with custom field delimiter2-0-e8d2b2e60551f69bfb44e555f5cff064 @@ -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/transform with custom field delimiter3-0-d4f4f471819345e9ce1964e281ea5289 b/sql/hive/src/test/resources/golden/transform with custom field delimiter3-0-d4f4f471819345e9ce1964e281ea5289 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform with custom field delimiter3-0-d4f4f471819345e9ce1964e281ea5289 @@ -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/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 60619f5d99578..4c53b10ba96e9 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 @@ -63,7 +63,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { sql("SHOW TABLES") } } - + createQueryTest("! operator", """ |SELECT a FROM ( @@ -329,6 +329,80 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("transform", "SELECT TRANSFORM (key) USING 'cat' AS (tKey) FROM src") + createQueryTest("schema-less transform", + """ + |SELECT TRANSFORM (key, value) USING 'cat' FROM src; + |SELECT TRANSFORM (*) USING 'cat' FROM src; + """.stripMargin) + + val delimiter = "'\t'" + + createQueryTest("transform with custom field delimiter", + s""" + |SELECT TRANSFORM (key) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} + |USING 'cat' AS (tKey) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src; + """.stripMargin.replaceAll("\n", " ")) + + createQueryTest("transform with custom field delimiter2", + s""" + |SELECT TRANSFORM (key, value) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} + |USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src; + """.stripMargin.replaceAll("\n", " ")) + + createQueryTest("transform with custom field delimiter3", + s""" + |SELECT TRANSFORM (*) ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} + |USING 'cat' ROW FORMAT DELIMITED FIELDS TERMINATED BY ${delimiter} FROM src; + """.stripMargin.replaceAll("\n", " ")) + + createQueryTest("transform with SerDe", + """ + |SELECT TRANSFORM (key, value) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + |USING 'cat' AS (tKey, tValue) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' FROM src; + """.stripMargin.replaceAll("\n", " ")) + + test("transform with SerDe2") { + + sql("CREATE TABLE small_src(key INT, value STRING)") + sql("INSERT OVERWRITE TABLE small_src SELECT key, value FROM src LIMIT 10") + + val expected = sql("SELECT key FROM small_src").collect().head + val res = sql( + """ + |SELECT TRANSFORM (key) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.avro.AvroSerDe' + |WITH SERDEPROPERTIES ('avro.schema.literal'='{"namespace": + |"testing.hive.avro.serde","name": "src","type": "record","fields": + |[{"name":"key","type":"int"}]}') USING 'cat' AS (tKey INT) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.avro.AvroSerDe' WITH SERDEPROPERTIES + |('avro.schema.literal'='{"namespace": "testing.hive.avro.serde","name": + |"src","type": "record","fields": [{"name":"key","type":"int"}]}') + |FROM small_src + """.stripMargin.replaceAll("\n", " ")).collect().head + + assert(expected(0) === res(0)) + } + + createQueryTest("transform with SerDe3", + """ + |SELECT TRANSFORM (*) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES + |('serialization.last.column.takes.rest'='true') USING 'cat' AS (tKey, tValue) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + |WITH SERDEPROPERTIES ('serialization.last.column.takes.rest'='true') FROM src; + """.stripMargin.replaceAll("\n", " ")) + + createQueryTest("transform with SerDe4", + """ + |SELECT TRANSFORM (*) ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' WITH SERDEPROPERTIES + |('serialization.last.column.takes.rest'='true') USING 'cat' ROW FORMAT SERDE + |'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%'") 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 c0b7741bc3e53..254919e8f6fdc 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 @@ -38,7 +38,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, Primitive 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} -import org.apache.hadoop.io.NullWritable +import org.apache.hadoop.io.{NullWritable, Writable} import org.apache.hadoop.mapred.InputFormat import org.apache.spark.sql.types.{Decimal, DecimalType} @@ -241,6 +241,10 @@ private[hive] object HiveShim { Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue()) } } + + def prepareWritable(w: Writable): Writable = { + w + } } 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 c04cda7bf1537..45ca59ae56a38 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 @@ -19,13 +19,14 @@ package org.apache.spark.sql.hive import java.util.{ArrayList => JArrayList} import java.util.Properties +import java.rmi.server.UID import scala.collection.JavaConversions._ import scala.language.implicitConversions import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.io.NullWritable +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} @@ -39,6 +40,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjec import org.apache.hadoop.hive.serde2.objectinspector.{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.{io => hadoopIo} import org.apache.spark.Logging @@ -395,10 +397,23 @@ private[hive] object HiveShim { Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue(), hdoi.precision(), hdoi.scale()) } } + + /* + * Bug introduced in hive-0.13. AvroGenericRecordWritable has a member recordReaderID that + * is needed to initialize before serialization. + */ + def prepareWritable(w: Writable): Writable = { + w match { + case w: AvroGenericRecordWritable => + w.setRecordReaderID(new UID()) + case _ => + } + w + } } /* - * Bug introdiced in hive-0.13. FileSinkDesc is serilizable, but its member path is not. + * 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) From e908322cd5991e6cbdaaafb8cd494759dac01225 Mon Sep 17 00:00:00 2001 From: Iulian Dragos Date: Mon, 2 Feb 2015 14:00:33 -0800 Subject: [PATCH 007/578] [SPARK-4631][streaming][FIX] Wait for a receiver to start before publishing test data. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This fixes two sources of non-deterministic failures in this test: - wait for a receiver to be up before pushing data through MQTT - gracefully handle the case where the MQTT client is overloaded. There’s a hard-coded limit of 10 in-flight messages, and this test may hit it. Instead of crashing, we retry sending the message. Both of these are needed to make the test pass reliably on my machine. Author: Iulian Dragos Closes #4270 from dragos/issue/fix-flaky-test-SPARK-4631 and squashes the following commits: f66c482 [Iulian Dragos] [SPARK-4631][streaming] Wait for a receiver to start before publishing test data. d408a8e [Iulian Dragos] Install callback before connecting to MQTT broker. --- .../streaming/mqtt/MQTTInputDStream.scala | 26 +++++++------- .../streaming/mqtt/MQTTStreamSuite.scala | 35 +++++++++++++++++-- 2 files changed, 46 insertions(+), 15 deletions(-) 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 77661f71ada21..1ef91dd49284f 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 @@ -55,14 +55,14 @@ class MQTTInputDStream( brokerUrl: String, topic: String, storageLevel: StorageLevel - ) extends ReceiverInputDStream[String](ssc_) with Logging { - + ) extends ReceiverInputDStream[String](ssc_) { + def getReceiver(): Receiver[String] = { new MQTTReceiver(brokerUrl, topic, storageLevel) } } -private[streaming] +private[streaming] class MQTTReceiver( brokerUrl: String, topic: String, @@ -72,21 +72,15 @@ class MQTTReceiver( def onStop() { } - + def onStart() { - // Set up persistence for messages + // Set up persistence for messages val persistence = new MemoryPersistence() // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance val client = new MqttClient(brokerUrl, MqttClient.generateClientId(), persistence) - // Connect to MqttBroker - client.connect() - - // Subscribe to Mqtt topic - client.subscribe(topic) - // Callback automatically triggers as and when new message arrives on specified topic val callback: MqttCallback = new MqttCallback() { @@ -103,7 +97,15 @@ class MQTTReceiver( } } - // Set up callback for MqttClient + // Set up callback for MqttClient. This needs to happen before + // connecting or subscribing, otherwise messages may be lost client.setCallback(callback) + + // Connect to MqttBroker + client.connect() + + // Subscribe to Mqtt topic + client.subscribe(topic) + } } 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 fe53a29cba0c9..e84adc088a680 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 @@ -18,6 +18,8 @@ package org.apache.spark.streaming.mqtt import java.net.{URI, ServerSocket} +import java.util.concurrent.CountDownLatch +import java.util.concurrent.TimeUnit import scala.concurrent.duration._ import scala.language.postfixOps @@ -32,6 +34,8 @@ import org.scalatest.concurrent.Eventually import org.apache.spark.streaming.{Milliseconds, StreamingContext} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.streaming.scheduler.StreamingListener +import org.apache.spark.streaming.scheduler.StreamingListenerReceiverStarted import org.apache.spark.SparkConf import org.apache.spark.util.Utils @@ -67,7 +71,7 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { val sendMessage = "MQTT demo for spark streaming" val receiveStream: ReceiverInputDStream[String] = MQTTUtils.createStream(ssc, "tcp:" + brokerUri, topic, StorageLevel.MEMORY_ONLY) - var receiveMessage: List[String] = List() + @volatile var receiveMessage: List[String] = List() receiveStream.foreachRDD { rdd => if (rdd.collect.length > 0) { receiveMessage = receiveMessage ::: List(rdd.first) @@ -75,6 +79,11 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { } } ssc.start() + + // wait for the receiver to start before publishing data, or we risk failing + // the test nondeterministically. See SPARK-4631 + waitForReceiverToStart() + publishData(sendMessage) eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { assert(sendMessage.equals(receiveMessage(0))) @@ -121,8 +130,14 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { val message: MqttMessage = new MqttMessage(data.getBytes("utf-8")) message.setQos(1) message.setRetained(true) - for (i <- 0 to 100) { - msgTopic.publish(message) + + for (i <- 0 to 10) { + try { + 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 + } } } } finally { @@ -131,4 +146,18 @@ class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { client = null } } + + /** + * Block until at least one receiver has started or timeout occurs. + */ + private def waitForReceiverToStart() = { + val latch = new CountDownLatch(1) + ssc.addStreamingListener(new StreamingListener { + override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { + latch.countDown() + } + }) + + assert(latch.await(10, TimeUnit.SECONDS), "Timeout waiting for receiver to start.") + } } From 2321dd1ef9b0ae70b686904fca4981549cc2d9b2 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 2 Feb 2015 14:00:14 -0800 Subject: [PATCH 008/578] [HOTFIX] Add jetty references to build for YARN module. --- yarn/pom.xml | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/yarn/pom.xml b/yarn/pom.xml index 7595549e4b6dc..6bdf9d2416823 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -58,6 +58,30 @@ org.apache.hadoop hadoop-client + + + + com.google.guava + guava + + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty + jetty-plus + + + org.eclipse.jetty + jetty-util + + + org.eclipse.jetty + jetty-http + + + org.apache.hadoop hadoop-yarn-server-tests From 52f5754f45370f98e577a2aa96accf3ee2e2c8e2 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 21 Jan 2015 14:38:14 -0800 Subject: [PATCH 009/578] Make sure only owner can read / write to directories created for the job. Whenever a directory is created by the utility method, immediately restrict its permissions so that only the owner has access to its contents. Signed-off-by: Josh Rosen --- .../org/apache/spark/HttpFileServer.scala | 2 +- .../scala/org/apache/spark/SparkEnv.scala | 2 +- .../spark/broadcast/HttpBroadcast.scala | 2 +- .../spark/storage/DiskBlockManager.scala | 34 ++------ .../scala/org/apache/spark/util/Utils.scala | 77 +++++++++++++------ python/pyspark/context.py | 3 +- .../apache/spark/streaming/JavaAPISuite.java | 4 +- 7 files changed, 69 insertions(+), 55 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HttpFileServer.scala b/core/src/main/scala/org/apache/spark/HttpFileServer.scala index 677c5e0f89d72..3f33332a81eaf 100644 --- a/core/src/main/scala/org/apache/spark/HttpFileServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpFileServer.scala @@ -36,7 +36,7 @@ private[spark] class HttpFileServer( var serverUri : String = null def initialize() { - baseDir = Utils.createTempDir() + baseDir = Utils.createTempDir(Utils.getLocalDir(conf), "httpd") fileDir = new File(baseDir, "files") jarDir = new File(baseDir, "jars") fileDir.mkdir() diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 1264a8126153b..f25db7f8de565 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -339,7 +339,7 @@ object SparkEnv extends Logging { // this is a temporary directory; in distributed mode, this is the executor's current working // directory. val sparkFilesDir: String = if (isDriver) { - Utils.createTempDir().getAbsolutePath + Utils.createTempDir(Utils.getLocalDir(conf), "userFiles").getAbsolutePath } else { "." } 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 31d6958c403b3..ea98051532a0a 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -151,7 +151,7 @@ private[broadcast] object HttpBroadcast extends Logging { } private def createServer(conf: SparkConf) { - broadcastDir = Utils.createTempDir(Utils.getLocalDir(conf)) + broadcastDir = Utils.createTempDir(Utils.getLocalDir(conf), "broadcast") val broadcastPort = conf.getInt("spark.broadcast.port", 0) server = new HttpServer(conf, broadcastDir, securityManager, broadcastPort, "HTTP broadcast server") 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 af05eb3ca69ce..53eaedacbf291 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -17,9 +17,8 @@ package org.apache.spark.storage +import java.util.UUID import java.io.{IOException, File} -import java.text.SimpleDateFormat -import java.util.{Date, Random, UUID} import org.apache.spark.{SparkConf, Logging} import org.apache.spark.executor.ExecutorExitCode @@ -37,7 +36,6 @@ import org.apache.spark.util.Utils private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkConf) extends Logging { - private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 private[spark] val subDirsPerLocalDir = blockManager.conf.getInt("spark.diskStore.subDirectories", 64) @@ -123,33 +121,15 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon } private def createLocalDirs(conf: SparkConf): Array[File] = { - val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss") Utils.getOrCreateLocalRootDirs(conf).flatMap { rootDir => - var foundLocalDir = false - var localDir: File = null - var localDirId: String = null - var tries = 0 - val rand = new Random() - while (!foundLocalDir && tries < MAX_DIR_CREATION_ATTEMPTS) { - tries += 1 - try { - localDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536)) - localDir = new File(rootDir, s"spark-local-$localDirId") - if (!localDir.exists) { - foundLocalDir = localDir.mkdirs() - } - } catch { - case e: Exception => - logWarning(s"Attempt $tries to create local dir $localDir failed", e) - } - } - if (!foundLocalDir) { - logError(s"Failed $MAX_DIR_CREATION_ATTEMPTS attempts to create local dir in $rootDir." + - " Ignoring this directory.") - None - } else { + try { + val localDir = Utils.createDirectory(rootDir, "blockmgr") logInfo(s"Created local directory at $localDir") Some(localDir) + } catch { + case e: IOException => + logError(s"Failed to create local dir in $rootDir. Ignoring this directory.", e) + None } } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 86ac307fc84ba..e0236d23c41ba 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -60,6 +60,8 @@ private[spark] object CallSite { private[spark] object Utils extends Logging { val random = new Random() + private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 + /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -246,13 +248,28 @@ private[spark] object Utils extends Logging { retval } + /** + * JDK equivalent of `chmod 700 file`. + * + * @param file the file whose permissions will be modified + * @return true if the permissions were successfully changed, false otherwise. + */ + def chmod700(file: File): Boolean = { + file.setReadable(false, false) && + file.setReadable(true, true) && + file.setWritable(false, false) && + file.setWritable(true, true) && + file.setExecutable(false, false) && + file.setExecutable(true, true) + } + /** * Create a directory inside the given parent directory. The directory is guaranteed to be * newly created, and is not marked for automatic deletion. */ - def createDirectory(root: String): File = { + def createDirectory(root: String, namePrefix: String = "spark"): File = { var attempts = 0 - val maxAttempts = 10 + val maxAttempts = MAX_DIR_CREATION_ATTEMPTS var dir: File = null while (dir == null) { attempts += 1 @@ -264,6 +281,11 @@ private[spark] object Utils extends Logging { dir = new File(root, "spark-" + UUID.randomUUID.toString) if (dir.exists() || !dir.mkdirs()) { dir = null + } else { + if (!chmod700(dir)) { + dir.delete() + dir = null + } } } catch { case e: SecurityException => dir = null; } } @@ -275,8 +297,10 @@ private[spark] object Utils extends Logging { * Create a temporary directory inside the given parent directory. The directory will be * automatically deleted when the VM shuts down. */ - def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { - val dir = createDirectory(root) + def createTempDir( + root: String = System.getProperty("java.io.tmpdir"), + namePrefix: String = "spark"): File = { + val dir = createDirectory(root, namePrefix) registerShutdownDeleteDir(dir) dir } @@ -599,26 +623,35 @@ private[spark] object Utils extends Logging { * If no directories could be created, this will return an empty list. */ private[spark] def getOrCreateLocalRootDirs(conf: SparkConf): Array[String] = { - val confValue = if (isRunningInYarnContainer(conf)) { + 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. - getYarnLocalDirs(conf) + // 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 { - Option(conf.getenv("SPARK_LOCAL_DIRS")).getOrElse( - conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) - } - val rootDirs = confValue.split(',') - logDebug(s"Getting/creating local root dirs at '$confValue'") - - rootDirs.flatMap { rootDir => - val localDir: File = new File(rootDir) - val foundLocalDir = localDir.exists || localDir.mkdirs() - if (!foundLocalDir) { - logError(s"Failed to create local root dir in $rootDir. Ignoring this directory.") - None - } else { - Some(rootDir) - } + // 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 + // permissions under each listed directory. + Option(conf.getenv("SPARK_LOCAL_DIRS")) + .getOrElse(conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) + .split(",") + .flatMap { root => + try { + val rootDir = new File(root) + if (rootDir.exists || rootDir.mkdirs()) { + Some(createDirectory(root).getAbsolutePath()) + } else { + logError(s"Failed to create dir in $root. Ignoring this directory.") + None + } + } catch { + case e: IOException => + logError(s"Failed to create local root dir in $root. Ignoring this directory.") + None + } + } + .toArray } } diff --git a/python/pyspark/context.py b/python/pyspark/context.py index c0dec16ac1b25..bf1f61c8504ed 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -192,7 +192,8 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, # Create a temporary directory inside spark.local.dir: local_dir = self._jvm.org.apache.spark.util.Utils.getLocalDir(self._jsc.sc().conf()) self._temp_dir = \ - self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir).getAbsolutePath() + self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir, "pyspark") \ + .getAbsolutePath() # profiling stats collected for each PythonRDD if self._conf.get("spark.python.profile", "false") == "true": 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 d4c40745658c2..2df8cf6a8a3df 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -1769,7 +1769,7 @@ public Iterable call(InputStream in) throws IOException { @SuppressWarnings("unchecked") @Test public void testTextFileStream() throws IOException { - File testDir = Utils.createTempDir(System.getProperty("java.io.tmpdir")); + File testDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark"); List> expected = fileTestPrepare(testDir); JavaDStream input = ssc.textFileStream(testDir.toString()); @@ -1782,7 +1782,7 @@ public void testTextFileStream() throws IOException { @SuppressWarnings("unchecked") @Test public void testFileStream() throws IOException { - File testDir = Utils.createTempDir(System.getProperty("java.io.tmpdir")); + File testDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "spark"); List> expected = fileTestPrepare(testDir); JavaPairInputDStream inputStream = ssc.fileStream( From bff65b5cca7ae0c6c49e6a04638d18104be4be7c Mon Sep 17 00:00:00 2001 From: Martin Weindel Date: Mon, 2 Feb 2015 13:46:18 -0800 Subject: [PATCH 010/578] Disabling Utils.chmod700 for Windows This patch makes Spark 1.2.1rc2 work again on Windows. Without it you get following log output on creating a Spark context: INFO org.apache.spark.SparkEnv:59 - Registering BlockManagerMaster ERROR org.apache.spark.util.Utils:75 - Failed to create local root dir in .... Ignoring this directory. ERROR org.apache.spark.storage.DiskBlockManager:75 - Failed to create any local dir. Author: Martin Weindel Author: mweindel Closes #4299 from MartinWeindel/branch-1.2 and squashes the following commits: 535cb7f [Martin Weindel] fixed last commit f17072e [Martin Weindel] moved condition to caller to avoid confusion on chmod700() return value 4de5e91 [Martin Weindel] reverted to unix line ends fe2740b [mweindel] moved comment ac4749c [mweindel] fixed chmod700 for Windows --- core/src/main/scala/org/apache/spark/util/Utils.scala | 4 +++- 1 file changed, 3 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 e0236d23c41ba..703b23add834b 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -282,7 +282,9 @@ private[spark] object Utils extends Logging { if (dir.exists() || !dir.mkdirs()) { dir = null } else { - if (!chmod700(dir)) { + // Restrict file permissions via chmod if available. + // For Windows this step is ignored. + if (!isWindows && !chmod700(dir)) { dir.delete() dir = null } From 5a5526164bdf9ecf1306d4570e816eb4df5cfd2b Mon Sep 17 00:00:00 2001 From: Jacek Lewandowski Date: Mon, 2 Feb 2015 14:07:19 -0800 Subject: [PATCH 011/578] SPARK-5425: Use synchronised methods in system properties to create SparkConf SPARK-5425: Fixed usages of system properties This patch fixes few problems caused by the fact that the Scala wrapper over system properties is not thread-safe and is basically invalid because it doesn't take into account the default values which could have been set in the properties object. The problem is fixed by modifying `Utils.getSystemProperties` method so that it uses `stringPropertyNames` method of the `Properties` class, which is thread-safe (internally it creates a defensive copy in a synchronized method) and returns keys of the properties which were set explicitly and which are defined as defaults. The other related problem, which is fixed here. was in `ResetSystemProperties` mix-in. It created a copy of the system properties in the wrong way. This patch also introduces a test case for thread-safeness of SparkConf creation. Refer to the discussion in https://github.com/apache/spark/pull/4220 for more details. Author: Jacek Lewandowski Closes #4222 from jacek-lewandowski/SPARK-5425-1.3 and squashes the following commits: 03da61b [Jacek Lewandowski] SPARK-5425: Modified Utils.getSystemProperties to return a map of all system properties - explicit + defaults 8faf2ea [Jacek Lewandowski] SPARK-5425: Use SerializationUtils to save properties in ResetSystemProperties trait 71aa572 [Jacek Lewandowski] SPARK-5425: Use synchronised methods in system properties to create SparkConf --- .../scala/org/apache/spark/SparkConf.scala | 5 ++-- .../scala/org/apache/spark/util/Utils.scala | 11 +++++--- .../org/apache/spark/SparkConfSuite.scala | 25 +++++++++++++++++++ .../spark/util/ResetSystemProperties.scala | 7 +++++- .../spark/examples/DriverSubmissionTest.scala | 4 ++- 5 files changed, 45 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index cd91c8f87547b..4d4c69d42da98 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -23,6 +23,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.LinkedHashSet import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.util.Utils /** * Configuration for a Spark application. Used to set various Spark parameters as key-value pairs. @@ -53,8 +54,8 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { if (loadDefaults) { // Load any spark.* system properties - for ((k, v) <- System.getProperties.asScala if k.startsWith("spark.")) { - set(k, v) + for ((key, value) <- Utils.getSystemProperties if key.startsWith("spark.")) { + set(key, value) } } 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 703b23add834b..31850b50bdba7 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1347,9 +1347,14 @@ private[spark] object Utils extends Logging { hashAbs } - /** Returns a copy of the system properties that is thread-safe to iterator over. */ - def getSystemProperties(): Map[String, String] = { - System.getProperties.clone().asInstanceOf[java.util.Properties].toMap[String, String] + /** Returns the system properties map that is thread-safe to iterator over. It gets the + * properties which have been set explicitly, as well as those for which only a default value + * has been defined. */ + def getSystemProperties: Map[String, String] = { + val sysProps = for (key <- System.getProperties.stringPropertyNames()) yield + (key, System.getProperty(key)) + + sysProps.toMap } /** diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 790976a5ac308..e08210ae60d17 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -17,6 +17,10 @@ package org.apache.spark +import java.util.concurrent.{TimeUnit, Executors} + +import scala.util.{Try, Random} + import org.scalatest.FunSuite import org.apache.spark.serializer.{KryoRegistrator, KryoSerializer} import org.apache.spark.util.ResetSystemProperties @@ -123,6 +127,27 @@ class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemPro assert(conf.get("spark.test.a.b.c") === "a.b.c") } + test("Thread safeness - SPARK-5425") { + import scala.collection.JavaConversions._ + val executor = Executors.newSingleThreadScheduledExecutor() + val sf = executor.scheduleAtFixedRate(new Runnable { + override def run(): Unit = + System.setProperty("spark.5425." + Random.nextInt(), Random.nextInt().toString) + }, 0, 1, TimeUnit.MILLISECONDS) + + try { + val t0 = System.currentTimeMillis() + while ((System.currentTimeMillis() - t0) < 1000) { + val conf = Try(new SparkConf(loadDefaults = true)) + assert(conf.isSuccess === true) + } + } finally { + executor.shutdownNow() + for (key <- System.getProperties.stringPropertyNames() if key.startsWith("spark.5425.")) + System.getProperties.remove(key) + } + } + test("register kryo classes through registerKryoClasses") { val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") diff --git a/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala b/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala index d4b92f33dd9e6..bad1aa99952cf 100644 --- a/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala +++ b/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala @@ -19,6 +19,7 @@ package org.apache.spark.util import java.util.Properties +import org.apache.commons.lang3.SerializationUtils import org.scalatest.{BeforeAndAfterEach, Suite} /** @@ -42,7 +43,11 @@ private[spark] trait ResetSystemProperties extends BeforeAndAfterEach { this: Su var oldProperties: Properties = null override def beforeEach(): Unit = { - oldProperties = new Properties(System.getProperties) + // we need SerializationUtils.clone instead of `new Properties(System.getProperties()` because + // the later way of creating a copy does not copy the properties but it initializes a new + // Properties object with the given properties as defaults. They are not recognized at all + // by standard Scala wrapper over Java Properties then. + oldProperties = SerializationUtils.clone(System.getProperties) super.beforeEach() } diff --git a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala index 65251e93190f0..e757283823fc3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala @@ -19,6 +19,8 @@ package org.apache.spark.examples import scala.collection.JavaConversions._ +import org.apache.spark.util.Utils + /** Prints out environmental information, sleeps, and then exits. Made to * test driver submission in the standalone scheduler. */ object DriverSubmissionTest { @@ -30,7 +32,7 @@ object DriverSubmissionTest { val numSecondsToSleep = args(0).toInt val env = System.getenv() - val properties = System.getProperties() + val properties = Utils.getSystemProperties println("Environment variables containing SPARK_TEST:") env.filter{case (k, v) => k.contains("SPARK_TEST")}.foreach(println) From 842d00032d0b09fb1f9cfc77359b77693e70a614 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 2 Feb 2015 14:34:48 -0800 Subject: [PATCH 012/578] [SPARK-5461] [graphx] Add isCheckpointed, getCheckpointedFiles methods to Graph Added the 2 methods to Graph and GraphImpl. Both make calls to the underlying vertex and edge RDDs. This is needed for another PR (for LDA): [https://github.com/apache/spark/pull/4047] Notes: * getCheckpointedFiles is plural and returns a Seq[String] instead of an Option[String]. * I attempted to test to make sure the methods returned the correct values after checkpointing. It did not work; I guess that checkpointing does not occur quickly enough? I noticed that there are not checkpointing tests for RDDs; is it just hard to test well? CC: rxin CC: mengxr (since related to LDA) Author: Joseph K. Bradley Closes #4253 from jkbradley/graphx-checkpoint and squashes the following commits: b680148 [Joseph K. Bradley] added class tag to firstParent call in VertexRDDImpl.isCheckpointed, though not needed to compile 250810e [Joseph K. Bradley] In EdgeRDDImple, VertexRDDImpl, added transient back to partitionsRDD, and made isCheckpointed check firstParent instead of partitionsRDD 695b7a3 [Joseph K. Bradley] changed partitionsRDD in EdgeRDDImpl, VertexRDDImpl to be non-transient cc00767 [Joseph K. Bradley] added overrides for isCheckpointed, getCheckpointFile in EdgeRDDImpl, VertexRDDImpl. The corresponding Graph methods now work. 188665f [Joseph K. Bradley] improved documentation 235738c [Joseph K. Bradley] Added isCheckpointed and getCheckpointFiles to Graph, GraphImpl --- .../main/scala/org/apache/spark/graphx/Graph.scala | 12 ++++++++++++ .../org/apache/spark/graphx/impl/EdgeRDDImpl.scala | 10 +++++++++- .../org/apache/spark/graphx/impl/GraphImpl.scala | 11 +++++++++++ .../org/apache/spark/graphx/impl/VertexRDDImpl.scala | 10 +++++++++- .../scala/org/apache/spark/graphx/GraphSuite.scala | 4 ++++ project/MimaExcludes.scala | 6 ++++++ 6 files changed, 51 insertions(+), 2 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index ab56580a3abc8..8494d06b1cdb7 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -104,6 +104,18 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab */ def checkpoint(): Unit + /** + * Return whether this Graph has been checkpointed or not. + * This returns true iff both the vertices RDD and edges RDD have been checkpointed. + */ + def isCheckpointed: Boolean + + /** + * Gets the name of the files to which this Graph was checkpointed. + * (The vertices RDD and edges RDD are checkpointed separately.) + */ + def getCheckpointFiles: Seq[String] + /** * Uncaches both vertices and edges of this graph. This is useful in iterative algorithms that * build a new graph in each iteration. 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 f1550ac2e18ad..6c35d7029e078 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 @@ -73,7 +73,15 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] ( override def checkpoint() = { partitionsRDD.checkpoint() } - + + override def isCheckpointed: Boolean = { + firstParent[(PartitionID, EdgePartition[ED, VD])].isCheckpointed + } + + override def getCheckpointFile: Option[String] = { + partitionsRDD.getCheckpointFile + } + /** The number of edges in the RDD. */ override def count(): Long = { partitionsRDD.map(_._2.size.toLong).reduce(_ + _) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index 3f4a900d5b601..90a74d23a26cc 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -70,6 +70,17 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( replicatedVertexView.edges.checkpoint() } + override def isCheckpointed: Boolean = { + vertices.isCheckpointed && replicatedVertexView.edges.isCheckpointed + } + + override def getCheckpointFiles: Seq[String] = { + Seq(vertices.getCheckpointFile, replicatedVertexView.edges.getCheckpointFile).flatMap { + case Some(path) => Seq(path) + case None => Seq() + } + } + override def unpersist(blocking: Boolean = true): Graph[VD, ED] = { unpersistVertices(blocking) replicatedVertexView.edges.unpersist(blocking) 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 9732c5b00c6d9..3e4968d6c0d6f 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 @@ -74,7 +74,15 @@ class VertexRDDImpl[VD] private[graphx] ( override def checkpoint() = { partitionsRDD.checkpoint() } - + + override def isCheckpointed: Boolean = { + firstParent[ShippableVertexPartition[VD]].isCheckpointed + } + + override def getCheckpointFile: Option[String] = { + partitionsRDD.getCheckpointFile + } + /** The number of vertices in the RDD. */ override def count(): Long = { partitionsRDD.map(_.size).reduce(_ + _) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index ed9876b8dc21c..59a57ba7a33f1 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -375,6 +375,8 @@ class GraphSuite extends FunSuite with LocalSparkContext { val ring = (0L to 100L).zip((1L to 99L) :+ 0L).map { case (a, b) => Edge(a, b, 1)} val rdd = sc.parallelize(ring) val graph = Graph.fromEdges(rdd, 1.0F) + assert(!graph.isCheckpointed) + assert(graph.getCheckpointFiles.size === 0) graph.checkpoint() graph.edges.map(_.attr).count() graph.vertices.map(_._2).count() @@ -383,6 +385,8 @@ class GraphSuite extends FunSuite with LocalSparkContext { val verticesDependencies = graph.vertices.partitionsRDD.dependencies assert(edgesDependencies.forall(_.rdd.isInstanceOf[CheckpointRDD[_]])) assert(verticesDependencies.forall(_.rdd.isInstanceOf[CheckpointRDD[_]])) + assert(graph.isCheckpointed) + assert(graph.getCheckpointFiles.size === 2) } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 14ba03ed4634b..45be1db9a5ebe 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -127,6 +127,12 @@ object MimaExcludes { // SPARK-5315 Spark Streaming Java API returns Scala DStream ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.streaming.api.java.JavaDStreamLike.reduceByWindow") + ) ++ Seq( + // SPARK-5461 Graph should have isCheckpointed, getCheckpointFiles methods + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.graphx.Graph.getCheckpointFiles"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.graphx.Graph.isCheckpointed") ) case v if v.startsWith("1.2") => From 830934976e8cf9e894bd3e5758fb941cad5d2f0b Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 2 Feb 2015 14:52:46 -0800 Subject: [PATCH 013/578] SPARK-5500. Document that feeding hadoopFile into a shuffle operation wi... ...ll cause problems Author: Sandy Ryza Closes #4293 from sryza/sandy-spark-5500 and squashes the following commits: e9ce742 [Sandy Ryza] Change to warning cc46e52 [Sandy Ryza] Add instructions and extend to NewHadoopRDD 6e1932a [Sandy Ryza] Throw exception on cache 0f6c4eb [Sandy Ryza] SPARK-5500. Document that feeding hadoopFile into a shuffle operation will cause problems --- .../scala/org/apache/spark/SparkContext.scala | 69 +++++++++++-------- .../org/apache/spark/rdd/HadoopRDD.scala | 12 +++- .../org/apache/spark/rdd/NewHadoopRDD.scala | 17 +++-- 3 files changed, 62 insertions(+), 36 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3c61c10820ba9..228076f01c841 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -687,9 +687,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * @param minPartitions Minimum number of Hadoop Splits to generate. * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. */ def hadoopRDD[K, V]( conf: JobConf, @@ -705,12 +706,13 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } /** Get an RDD for a Hadoop file with an arbitrary InputFormat - * - * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. - * */ + * + * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. + */ def hadoopFile[K, V]( path: String, inputFormatClass: Class[_ <: InputFormat[K, V]], @@ -741,9 +743,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * }}} * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. */ def hadoopFile[K, V, F <: InputFormat[K, V]] (path: String, minPartitions: Int) @@ -764,9 +767,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * }}} * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. */ def hadoopFile[K, V, F <: InputFormat[K, V]](path: String) (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = @@ -788,9 +792,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * and extra configuration options to pass to the input format. * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. */ def newAPIHadoopFile[K, V, F <: NewInputFormat[K, V]]( path: String, @@ -810,9 +815,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * and extra configuration options to pass to the input format. * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. */ def newAPIHadoopRDD[K, V, F <: NewInputFormat[K, V]]( conf: Configuration = hadoopConfiguration, @@ -826,9 +832,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli /** Get an RDD for a Hadoop SequenceFile with given key and value types. * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. */ def sequenceFile[K, V](path: String, keyClass: Class[K], @@ -843,9 +850,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli /** Get an RDD for a Hadoop SequenceFile with given key and value types. * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. * */ def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V]): RDD[(K, V)] = { assertNotStopped() @@ -869,9 +877,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * allow it to figure out the Writable class to use in the subclass case. * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each - * record, directly caching the returned RDD will create many references to the same object. - * If you plan to directly cache Hadoop writable objects, you should first copy them using - * a `map` function. + * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle + * operation will create many references to the same object. + * If you plan to directly cache, sort, or aggregate Hadoop writable objects, you should first + * copy them using a `map` function. */ def sequenceFile[K, V] (path: String, minPartitions: Int = defaultMinPartitions) 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 c3e3931042de2..89adddcf0ac36 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -42,10 +42,11 @@ import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.{DataReadMethod, InputMetrics} +import org.apache.spark.executor.DataReadMethod import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD import org.apache.spark.util.{NextIterator, Utils} import org.apache.spark.scheduler.{HostTaskLocation, HDFSCacheTaskLocation} +import org.apache.spark.storage.StorageLevel /** * A Spark split class that wraps around a Hadoop InputSplit. @@ -308,6 +309,15 @@ class HadoopRDD[K, V]( // Do nothing. Hadoop RDD should not be checkpointed. } + override def persist(storageLevel: StorageLevel): this.type = { + if (storageLevel.deserialized) { + logWarning("Caching NewHadoopRDDs as deserialized objects usually leads to undesired" + + " behavior because Hadoop's RecordReader reuses the same Writable object for all records." + + " Use a map transformation to make copies of the records.") + } + super.persist(storageLevel) + } + def getConf: Configuration = getJobConf() } 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 d86f95ac3e485..44b9ffd2a53fd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -29,16 +29,13 @@ import org.apache.hadoop.mapreduce.lib.input.{CombineFileSplit, FileSplit} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.input.WholeTextFileInputFormat -import org.apache.spark.InterruptibleIterator -import org.apache.spark.Logging -import org.apache.spark.Partition -import org.apache.spark.SerializableWritable -import org.apache.spark.{SparkContext, TaskContext} +import org.apache.spark._ import org.apache.spark.executor.DataReadMethod import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.storage.StorageLevel private[spark] class NewHadoopPartition( rddId: Int, @@ -211,6 +208,16 @@ class NewHadoopRDD[K, V]( locs.getOrElse(split.getLocations.filter(_ != "localhost")) } + override def persist(storageLevel: StorageLevel): this.type = { + if (storageLevel.deserialized) { + logWarning("Caching NewHadoopRDDs as deserialized objects usually leads to undesired" + + " behavior because Hadoop's RecordReader reuses the same Writable object for all records." + + " Use a map transformation to make copies of the records.") + } + super.persist(storageLevel) + } + + def getConf: Configuration = confBroadcast.value.value } From 1646f89d967913ee1f231d9606f8502d13c25804 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 2 Feb 2015 15:49:22 -0800 Subject: [PATCH 014/578] [SPARK-4508] [SQL] build native date type to conform behavior to Hive Store daysSinceEpoch as an Int value(4 bytes) to represent DateType, instead of using java.sql.Date(8 bytes as Long) in catalyst row. This ensures the same comparison behavior of Hive and Catalyst. Subsumes #3381 I thinks there are already some tests in JavaSQLSuite, and for python it will not affect python's datetime class. Author: Daoyuan Wang Closes #3732 from adrian-wang/datenative and squashes the following commits: 0ed0fdc [Daoyuan Wang] fix test data a2fdd4e [Daoyuan Wang] getDate c37832b [Daoyuan Wang] row to catalyst f0005b1 [Daoyuan Wang] add date in sql parser and java type conversion 024c9a6 [Daoyuan Wang] clean some import order d6715fc [Daoyuan Wang] refactoring Date as Primitive Int internally 374abd5 [Daoyuan Wang] spark native date type support --- .../main/scala/org/apache/spark/sql/Row.scala | 2 +- .../spark/sql/catalyst/ScalaReflection.scala | 9 +-- .../apache/spark/sql/catalyst/SqlParser.scala | 2 + .../spark/sql/catalyst/expressions/Cast.scala | 53 +++++++--------- .../expressions/codegen/CodeGenerator.scala | 3 + .../sql/catalyst/expressions/literals.scala | 2 +- .../apache/spark/sql/types/DateUtils.scala | 60 +++++++++++++++++++ .../apache/spark/sql/types/dataTypes.scala | 12 ++-- .../ExpressionEvaluationSuite.scala | 28 ++++----- .../spark/sql/types/DataTypeSuite.scala | 2 +- .../spark/sql/columnar/ColumnStats.scala | 19 +----- .../spark/sql/columnar/ColumnType.scala | 13 ++-- .../spark/sql/execution/pythonUdfs.scala | 4 +- .../org/apache/spark/sql/json/JsonRDD.scala | 6 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 7 +++ .../sql/ScalaReflectionRelationSuite.scala | 3 +- .../spark/sql/columnar/ColumnStatsSuite.scala | 2 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 6 +- .../sql/columnar/ColumnarTestUtils.scala | 6 +- .../org/apache/spark/sql/json/JsonSuite.scala | 7 ++- .../execution/HiveCompatibilitySuite.scala | 1 + .../apache/spark/sql/hive/HiveContext.scala | 4 +- .../spark/sql/hive/HiveInspectors.scala | 11 +++- .../apache/spark/sql/hive/TableReader.scala | 3 +- ...te cast-0-a7cd69b80c77a771a2c955db666be53d | 1 + ... test 1-0-bde89be08a12361073ff658fef768b7e | 1 + ... test 2-0-dc1b267f1d79d49e6675afe4fd2a34a5 | 1 + .../date_1-0-50131c0ba7b7a6b65c789a5a8497bada | 1 + ...date_1-1-23edf29bf7376c70d5ecf12720f4b1eb} | 0 ...ate_1-10-df16364a220ff96a6ea1cd478cbc1d0b} | 0 ...ate_1-11-d964bec7e5632091ab5cb6f6786dbbf9} | 0 ...ate_1-12-480c5f024a28232b7857be327c992509} | 0 ...ate_1-13-4c0ed7fcb75770d8790575b586bf14f4} | 0 ...date_1-14-44fc74c1993062c0a9522199ff27fea} | 0 ...ate_1-15-4855a66124b16d1d0d003235995ac06b} | 0 ...ate_1-16-8bc190dba0f641840b5e1e198a14c55b} | 0 ...ate_1-17-23edf29bf7376c70d5ecf12720f4b1eb} | 0 ...date_1-2-4ebe3571c13a8b0c03096fbd972b7f1b} | 0 ... date_1-3-26b5c291400dfde455b3c1b878b71d0} | 0 ...date_1-4-df16364a220ff96a6ea1cd478cbc1d0b} | 0 ...date_1-5-d964bec7e5632091ab5cb6f6786dbbf9} | 0 ...date_1-6-559d01fb0b42c42f0c4927fa0f9deac4} | 0 ...date_1-7-df16364a220ff96a6ea1cd478cbc1d0b} | 0 ...date_1-8-d964bec7e5632091ab5cb6f6786dbbf9} | 0 ...date_1-9-8306558e0eabe936ac33dabaaa17fea4} | 0 .../spark/sql/hive/HiveInspectorSuite.scala | 4 +- .../sql/hive/execution/HiveQuerySuite.scala | 26 +++++++- .../org/apache/spark/sql/hive/Shim12.scala | 2 +- .../org/apache/spark/sql/hive/Shim13.scala | 2 +- 49 files changed, 191 insertions(+), 112 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala create mode 100644 sql/hive/src/test/resources/golden/Date cast-0-a7cd69b80c77a771a2c955db666be53d create mode 100644 sql/hive/src/test/resources/golden/Date comparison test 1-0-bde89be08a12361073ff658fef768b7e create mode 100644 sql/hive/src/test/resources/golden/Date comparison test 2-0-dc1b267f1d79d49e6675afe4fd2a34a5 create mode 100644 sql/hive/src/test/resources/golden/date_1-0-50131c0ba7b7a6b65c789a5a8497bada rename sql/hive/src/test/resources/golden/{date_1-0-23edf29bf7376c70d5ecf12720f4b1eb => date_1-1-23edf29bf7376c70d5ecf12720f4b1eb} (100%) rename sql/hive/src/test/resources/golden/{date_1-3-df16364a220ff96a6ea1cd478cbc1d0b => date_1-10-df16364a220ff96a6ea1cd478cbc1d0b} (100%) rename sql/hive/src/test/resources/golden/{date_1-10-d964bec7e5632091ab5cb6f6786dbbf9 => date_1-11-d964bec7e5632091ab5cb6f6786dbbf9} (100%) rename sql/hive/src/test/resources/golden/{date_1-11-480c5f024a28232b7857be327c992509 => date_1-12-480c5f024a28232b7857be327c992509} (100%) rename sql/hive/src/test/resources/golden/{date_1-12-4c0ed7fcb75770d8790575b586bf14f4 => date_1-13-4c0ed7fcb75770d8790575b586bf14f4} (100%) rename sql/hive/src/test/resources/golden/{date_1-13-44fc74c1993062c0a9522199ff27fea => date_1-14-44fc74c1993062c0a9522199ff27fea} (100%) rename sql/hive/src/test/resources/golden/{date_1-14-4855a66124b16d1d0d003235995ac06b => date_1-15-4855a66124b16d1d0d003235995ac06b} (100%) rename sql/hive/src/test/resources/golden/{date_1-15-8bc190dba0f641840b5e1e198a14c55b => date_1-16-8bc190dba0f641840b5e1e198a14c55b} (100%) rename sql/hive/src/test/resources/golden/{date_1-1-4ebe3571c13a8b0c03096fbd972b7f1b => date_1-17-23edf29bf7376c70d5ecf12720f4b1eb} (100%) rename sql/hive/src/test/resources/golden/{date_1-16-23edf29bf7376c70d5ecf12720f4b1eb => date_1-2-4ebe3571c13a8b0c03096fbd972b7f1b} (100%) rename sql/hive/src/test/resources/golden/{date_1-2-abdce0c0d14d3fc7441b7c134b02f99a => date_1-3-26b5c291400dfde455b3c1b878b71d0} (100%) rename sql/hive/src/test/resources/golden/{date_1-6-df16364a220ff96a6ea1cd478cbc1d0b => date_1-4-df16364a220ff96a6ea1cd478cbc1d0b} (100%) rename sql/hive/src/test/resources/golden/{date_1-4-d964bec7e5632091ab5cb6f6786dbbf9 => date_1-5-d964bec7e5632091ab5cb6f6786dbbf9} (100%) rename sql/hive/src/test/resources/golden/{date_1-5-5e70fc74158fbfca38134174360de12d => date_1-6-559d01fb0b42c42f0c4927fa0f9deac4} (100%) rename sql/hive/src/test/resources/golden/{date_1-9-df16364a220ff96a6ea1cd478cbc1d0b => date_1-7-df16364a220ff96a6ea1cd478cbc1d0b} (100%) rename sql/hive/src/test/resources/golden/{date_1-7-d964bec7e5632091ab5cb6f6786dbbf9 => date_1-8-d964bec7e5632091ab5cb6f6786dbbf9} (100%) rename sql/hive/src/test/resources/golden/{date_1-8-1d5c58095cd52ea539d869f2ab1ab67d => date_1-9-8306558e0eabe936ac33dabaaa17fea4} (100%) 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 41bb4f012f2e1..3a70d25534968 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 object Row { /** 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 e0db587efb08d..8e79e532ca564 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 @@ -17,14 +17,13 @@ package org.apache.spark.sql.catalyst -import java.sql.{Date, Timestamp} +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.plans.logical.LocalRelation import org.apache.spark.sql.types._ - /** * A default version of ScalaReflection that uses the runtime universe. */ @@ -72,6 +71,7 @@ trait ScalaReflection { }.toArray) case (d: BigDecimal, _) => Decimal(d) case (d: java.math.BigDecimal, _) => Decimal(d) + case (d: java.sql.Date, _) => DateUtils.fromJavaDate(d) case (other, _) => other } @@ -85,6 +85,7 @@ trait ScalaReflection { } case (r: Row, s: StructType) => convertRowToScala(r, s) case (d: Decimal, _: DecimalType) => d.toJavaBigDecimal + case (i: Int, DateType) => DateUtils.toJavaDate(i) case (other, _) => other } @@ -159,7 +160,7 @@ trait ScalaReflection { 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[Date] => Schema(DateType, nullable = true) + case t if t <:< typeOf[java.sql.Date] => Schema(DateType, nullable = true) case t if t <:< typeOf[BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) case t if t <:< typeOf[java.math.BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) case t if t <:< typeOf[Decimal] => Schema(DecimalType.Unlimited, nullable = true) @@ -191,7 +192,7 @@ trait ScalaReflection { case obj: LongType.JvmType => LongType case obj: FloatType.JvmType => FloatType case obj: DoubleType.JvmType => DoubleType - case obj: DateType.JvmType => DateType + case obj: java.sql.Date => DateType case obj: java.math.BigDecimal => DecimalType.Unlimited case obj: Decimal => DecimalType.Unlimited case obj: TimestampType.JvmType => TimestampType 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 594a423146d77..2ce8be8e24e85 100755 --- 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 @@ -52,6 +52,7 @@ 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") @@ -383,6 +384,7 @@ class SqlParser extends AbstractSparkSQLParser { | DOUBLE ^^^ DoubleType | fixedDecimalType | DECIMAL ^^^ DecimalType.Unlimited + | DATE ^^^ DateType ) protected lazy val fixedDecimalType: Parser[DataType] = 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 ece5ee73618cb..b1bc858478ee1 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 @@ -113,7 +113,7 @@ 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[Date](_, dateToString) + case DateType => buildCast[Int](_, d => DateUtils.toString(d)) case TimestampType => buildCast[Timestamp](_, timestampToString) case _ => buildCast[Any](_, _.toString) } @@ -131,7 +131,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w buildCast[Timestamp](_, t => t.getTime() != 0 || t.getNanos() != 0) case DateType => // Hive would return null when cast from date to boolean - buildCast[Date](_, d => null) + buildCast[Int](_, d => null) case LongType => buildCast[Long](_, _ != 0) case IntegerType => @@ -171,7 +171,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case ByteType => buildCast[Byte](_, b => new Timestamp(b)) case DateType => - buildCast[Date](_, d => new Timestamp(d.getTime)) + buildCast[Int](_, d => new Timestamp(DateUtils.toJavaDate(d).getTime)) // TimestampWritable.decimalToTimestamp case DecimalType() => buildCast[Decimal](_, d => decimalToTimestamp(d)) @@ -224,37 +224,24 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w } } - // Converts Timestamp to string according to Hive TimestampWritable convention - private[this] def timestampToDateString(ts: Timestamp): String = { - Cast.threadLocalDateFormat.get.format(ts) - } - // DateConverter private[this] def castToDate(from: DataType): Any => Any = from match { case StringType => buildCast[String](_, s => - try Date.valueOf(s) catch { case _: java.lang.IllegalArgumentException => null }) + try DateUtils.fromJavaDate(Date.valueOf(s)) + catch { case _: java.lang.IllegalArgumentException => null } + ) case TimestampType => // throw valid precision more than seconds, according to Hive. // Timestamp.nanos is in 0 to 999,999,999, no more than a second. - buildCast[Timestamp](_, t => new Date(Math.floor(t.getTime / 1000.0).toLong * 1000)) + buildCast[Timestamp](_, t => DateUtils.millisToDays(t.getTime)) // Hive throws this exception as a Semantic Exception - // It is never possible to compare result when hive return with exception, so we can return null + // It is never possible to compare result when hive return with exception, + // so we can return null // NULL is more reasonable here, since the query itself obeys the grammar. case _ => _ => null } - // Date cannot be cast to long, according to hive - private[this] def dateToLong(d: Date) = null - - // Date cannot be cast to double, according to hive - private[this] def dateToDouble(d: Date) = null - - // Converts Date to string according to Hive DateWritable convention - private[this] def dateToString(d: Date): String = { - Cast.threadLocalDateFormat.get.format(d) - } - // LongConverter private[this] def castToLong(from: DataType): Any => Any = from match { case StringType => @@ -264,7 +251,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => if (b) 1L else 0L) case DateType => - buildCast[Date](_, d => dateToLong(d)) + buildCast[Int](_, d => null) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t)) case x: NumericType => @@ -280,7 +267,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => if (b) 1 else 0) case DateType => - buildCast[Date](_, d => dateToLong(d)) + buildCast[Int](_, d => null) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toInt) case x: NumericType => @@ -296,7 +283,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => if (b) 1.toShort else 0.toShort) case DateType => - buildCast[Date](_, d => dateToLong(d)) + buildCast[Int](_, d => null) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toShort) case x: NumericType => @@ -312,7 +299,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => if (b) 1.toByte else 0.toByte) case DateType => - buildCast[Date](_, d => dateToLong(d)) + buildCast[Int](_, d => null) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toByte) case x: NumericType => @@ -342,7 +329,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => changePrecision(if (b) Decimal(1) else Decimal(0), target)) case DateType => - buildCast[Date](_, d => null) // date can't cast to decimal in Hive + buildCast[Int](_, d => null) // date can't cast to decimal in Hive case TimestampType => // Note that we lose precision here. buildCast[Timestamp](_, t => changePrecision(Decimal(timestampToDouble(t)), target)) @@ -367,7 +354,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => if (b) 1d else 0d) case DateType => - buildCast[Date](_, d => dateToDouble(d)) + buildCast[Int](_, d => null) case TimestampType => buildCast[Timestamp](_, t => timestampToDouble(t)) case x: NumericType => @@ -383,7 +370,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => if (b) 1f else 0f) case DateType => - buildCast[Date](_, d => dateToDouble(d)) + buildCast[Int](_, d => null) case TimestampType => buildCast[Timestamp](_, t => timestampToDouble(t).toFloat) case x: NumericType => @@ -442,16 +429,16 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w object Cast { // `SimpleDateFormat` is not thread-safe. - private[sql] val threadLocalDateFormat = new ThreadLocal[DateFormat] { + private[sql] val threadLocalTimestampFormat = new ThreadLocal[DateFormat] { override def initialValue() = { - new SimpleDateFormat("yyyy-MM-dd") + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") } } // `SimpleDateFormat` is not thread-safe. - private[sql] val threadLocalTimestampFormat = new ThreadLocal[DateFormat] { + private[sql] val threadLocalDateFormat = new ThreadLocal[DateFormat] { override def initialValue() = { - new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + new SimpleDateFormat("yyyy-MM-dd") } } } 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 4cae5c4718683..1f80d84b744a1 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 @@ -246,6 +246,9 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin new String(${eval.primitiveTerm}.asInstanceOf[Array[Byte]]) """.children + case Cast(child @ DateType(), StringType) => + child.castOrNull(c => q"org.apache.spark.sql.types.DateUtils.toString($c)", StringType) + case Cast(child @ NumericType(), IntegerType) => child.castOrNull(c => q"$c.toInt", IntegerType) 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 5b389aad7a85d..97bb96f48e2c7 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 @@ -35,7 +35,7 @@ object Literal { case d: java.math.BigDecimal => Literal(Decimal(d), DecimalType.Unlimited) case d: Decimal => Literal(d, DecimalType.Unlimited) case t: Timestamp => Literal(t, TimestampType) - case d: Date => Literal(d, DateType) + case d: Date => Literal(DateUtils.fromJavaDate(d), DateType) case a: Array[Byte] => Literal(a, BinaryType) case null => Literal(null, NullType) } 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 new file mode 100644 index 0000000000000..8a1a3b81b3d2c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.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.sql.types + +import java.sql.Date +import java.util.{Calendar, TimeZone} + +import org.apache.spark.sql.catalyst.expressions.Cast + +/** + * helper function to convert between Int value of days since 1970-01-01 and java.sql.Date + */ +object DateUtils { + private val MILLIS_PER_DAY = 86400000 + + // Java TimeZone has no mention of thread safety. Use thread local instance to be safe. + private val LOCAL_TIMEZONE = new ThreadLocal[TimeZone] { + override protected def initialValue: TimeZone = { + Calendar.getInstance.getTimeZone + } + } + + private def javaDateToDays(d: Date): Int = { + millisToDays(d.getTime) + } + + def millisToDays(millisLocal: Long): Int = { + ((millisLocal + LOCAL_TIMEZONE.get().getOffset(millisLocal)) / MILLIS_PER_DAY).toInt + } + + private def toMillisSinceEpoch(days: Int): Long = { + val millisUtc = days.toLong * MILLIS_PER_DAY + millisUtc - LOCAL_TIMEZONE.get().getOffset(millisUtc) + } + + def fromJavaDate(date: java.sql.Date): Int = { + javaDateToDays(date) + } + + def toJavaDate(daysSinceEpoch: Int): java.sql.Date = { + new java.sql.Date(toMillisSinceEpoch(daysSinceEpoch)) + } + + def toString(days: Int): String = Cast.threadLocalDateFormat.get.format(toJavaDate(days)) +} 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 6ab99aa38877f..8ca0769fac287 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.types -import java.sql.{Date, Timestamp} +import java.sql.Timestamp import scala.math.Numeric.{FloatAsIfIntegral, DoubleAsIfIntegral} import scala.reflect.ClassTag @@ -387,18 +387,16 @@ case object TimestampType extends NativeType { */ @DeveloperApi case object DateType extends NativeType { - private[sql] type JvmType = Date + private[sql] type JvmType = Int @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - private[sql] val ordering = new Ordering[JvmType] { - def compare(x: Date, y: Date) = x.compareTo(y) - } + private[sql] val ordering = implicitly[Ordering[JvmType]] /** - * The default size of a value of the DateType is 8 bytes. + * The default size of a value of the DateType is 4 bytes. */ - override def defaultSize: Int = 8 + override def defaultSize: Int = 4 } 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 37e64adeea853..25d1c105a00a6 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 @@ -303,6 +303,7 @@ class ExpressionEvaluationSuite extends FunSuite { val sd = "1970-01-01" val d = Date.valueOf(sd) + val zts = sd + " 00:00:00" val sts = sd + " 00:00:02" val nts = sts + ".1" val ts = Timestamp.valueOf(nts) @@ -319,14 +320,14 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble) checkEvaluation(Cast(Literal(sd) cast DateType, StringType), sd) - checkEvaluation(Cast(Literal(d) cast StringType, DateType), d) + checkEvaluation(Cast(Literal(d) cast StringType, DateType), 0) checkEvaluation(Cast(Literal(nts) cast TimestampType, StringType), nts) checkEvaluation(Cast(Literal(ts) cast StringType, TimestampType), ts) // all convert to string type to check checkEvaluation( Cast(Cast(Literal(nts) cast TimestampType, DateType), StringType), sd) checkEvaluation( - Cast(Cast(Literal(ts) cast DateType, TimestampType), StringType), sts) + Cast(Cast(Literal(ts) cast DateType, TimestampType), StringType), zts) checkEvaluation(Cast("abdef" cast BinaryType, StringType), "abdef") @@ -377,8 +378,8 @@ class ExpressionEvaluationSuite extends FunSuite { } test("date") { - val d1 = Date.valueOf("1970-01-01") - val d2 = Date.valueOf("1970-01-02") + val d1 = DateUtils.fromJavaDate(Date.valueOf("1970-01-01")) + val d2 = DateUtils.fromJavaDate(Date.valueOf("1970-01-02")) checkEvaluation(Literal(d1) < Literal(d2), true) } @@ -459,22 +460,21 @@ class ExpressionEvaluationSuite extends FunSuite { test("date casting") { val d = Date.valueOf("1970-01-01") - checkEvaluation(Cast(d, ShortType), null) - checkEvaluation(Cast(d, IntegerType), null) - checkEvaluation(Cast(d, LongType), null) - checkEvaluation(Cast(d, FloatType), null) - checkEvaluation(Cast(d, DoubleType), null) - checkEvaluation(Cast(d, DecimalType.Unlimited), null) - checkEvaluation(Cast(d, DecimalType(10, 2)), null) - checkEvaluation(Cast(d, StringType), "1970-01-01") - checkEvaluation(Cast(Cast(d, TimestampType), StringType), "1970-01-01 00:00:00") + checkEvaluation(Cast(Literal(d), ShortType), null) + checkEvaluation(Cast(Literal(d), IntegerType), null) + checkEvaluation(Cast(Literal(d), LongType), null) + checkEvaluation(Cast(Literal(d), FloatType), null) + checkEvaluation(Cast(Literal(d), DoubleType), null) + checkEvaluation(Cast(Literal(d), DecimalType.Unlimited), null) + checkEvaluation(Cast(Literal(d), DecimalType(10, 2)), null) + checkEvaluation(Cast(Literal(d), StringType), "1970-01-01") + checkEvaluation(Cast(Cast(Literal(d), TimestampType), StringType), "1970-01-01 00:00:00") } test("timestamp casting") { val millis = 15 * 1000 + 2 val seconds = millis * 1000 + 2 val ts = new Timestamp(millis) - val ts1 = new Timestamp(15 * 1000) // a timestamp without the milliseconds part val tss = new Timestamp(seconds) checkEvaluation(Cast(ts, ShortType), 15) checkEvaluation(Cast(ts, IntegerType), 15) 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 c147be9f6b1ae..7bcd6687d11a1 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 @@ -106,7 +106,7 @@ class DataTypeSuite extends FunSuite { checkDefaultSize(DoubleType, 8) checkDefaultSize(DecimalType(10, 5), 4096) checkDefaultSize(DecimalType.Unlimited, 4096) - checkDefaultSize(DateType, 8) + checkDefaultSize(DateType, 4) checkDefaultSize(TimestampType, 8) checkDefaultSize(StringType, 4096) checkDefaultSize(BinaryType, 4096) 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 391b3dae5c8ce..cad0667b46435 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.columnar -import java.sql.{Date, Timestamp} +import java.sql.Timestamp import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.{AttributeMap, Attribute, AttributeReference} @@ -215,22 +215,7 @@ private[sql] class StringColumnStats extends ColumnStats { def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } -private[sql] class DateColumnStats extends ColumnStats { - protected var upper: Date = null - protected var lower: Date = null - - override def gatherStats(row: Row, ordinal: Int) { - super.gatherStats(row, ordinal) - if (!row.isNullAt(ordinal)) { - val value = row(ordinal).asInstanceOf[Date] - if (upper == null || value.compareTo(upper) > 0) upper = value - if (lower == null || value.compareTo(lower) < 0) lower = value - sizeInBytes += DATE.defaultSize - } - } - - def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) -} +private[sql] class DateColumnStats extends IntColumnStats private[sql] class TimestampColumnStats extends ColumnStats { protected var upper: Timestamp = null 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 fcf2faa0914c0..db5bc0de363c7 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 @@ -335,21 +335,20 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { } } -private[sql] object DATE extends NativeColumnType(DateType, 8, 8) { +private[sql] object DATE extends NativeColumnType(DateType, 8, 4) { override def extract(buffer: ByteBuffer) = { - val date = new Date(buffer.getLong()) - date + buffer.getInt } - override def append(v: Date, buffer: ByteBuffer): Unit = { - buffer.putLong(v.getTime) + override def append(v: Int, buffer: ByteBuffer): Unit = { + buffer.putInt(v) } override def getField(row: Row, ordinal: Int) = { - row(ordinal).asInstanceOf[Date] + row(ordinal).asInstanceOf[Int] } - override def setField(row: MutableRow, ordinal: Int, value: Date): Unit = { + def setField(row: MutableRow, ordinal: Int, value: Int): Unit = { row(ordinal) = value } } 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 b85021acc9d4c..3a2f8d75dac5e 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 @@ -135,6 +135,8 @@ object EvaluatePython { case (ud, udt: UserDefinedType[_]) => toJava(udt.serialize(ud), udt.sqlType) + case (date: Int, DateType) => DateUtils.toJavaDate(date) + // Pyrolite can handle Timestamp and Decimal case (other, _) => other } @@ -171,7 +173,7 @@ object EvaluatePython { }): Row case (c: java.util.Calendar, DateType) => - new java.sql.Date(c.getTime().getTime()) + DateUtils.fromJavaDate(new java.sql.Date(c.getTime().getTime())) case (c: java.util.Calendar, TimestampType) => new java.sql.Timestamp(c.getTime().getTime()) 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 9171939f7e8f7..33ce71b51b213 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 @@ -377,10 +377,12 @@ private[sql] object JsonRDD extends Logging { } } - private def toDate(value: Any): Date = { + private def toDate(value: Any): Int = { value match { // only support string as date - case value: java.lang.String => new Date(DataTypeConversions.stringToTime(value).getTime) + case value: java.lang.String => + DateUtils.millisToDays(DataTypeConversions.stringToTime(value).getTime) + case value: java.sql.Date => DateUtils.fromJavaDate(value) } } 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 d82c34316cefa..a7f6a50a04fbd 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 @@ -296,6 +296,13 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { mapData.collect().take(1).map(Row.fromTuple).toSeq) } + test("date row") { + checkAnswer(sql( + """select cast("2015-01-28" as date) from testData limit 1"""), + Row(java.sql.Date.valueOf("2015-01-28")) + ) + } + 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"), 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 a015884bae282..f26fcc0385b68 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 @@ -83,7 +83,8 @@ 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(12345), new Timestamp(12345), Seq(1,2,3))) + new java.math.BigDecimal(1), new Date(70, 0, 1), // This is 1970-01-01 + new Timestamp(12345), Seq(1,2,3))) } test("query case class RDD with nulls") { 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 be2b34de077c9..581fccf8ee613 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 @@ -30,7 +30,7 @@ class ColumnStatsSuite extends FunSuite { testColumnStats(classOf[FloatColumnStats], FLOAT, Row(Float.MaxValue, Float.MinValue, 0)) testColumnStats(classOf[DoubleColumnStats], DOUBLE, Row(Double.MaxValue, Double.MinValue, 0)) testColumnStats(classOf[StringColumnStats], STRING, Row(null, null, 0)) - testColumnStats(classOf[DateColumnStats], DATE, Row(null, null, 0)) + testColumnStats(classOf[DateColumnStats], DATE, Row(Int.MaxValue, Int.MinValue, 0)) testColumnStats(classOf[TimestampColumnStats], TIMESTAMP, Row(null, null, 0)) def testColumnStats[T <: NativeType, U <: ColumnStats]( 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 87e608a8853dc..9ce845912f1c7 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer -import java.sql.{Date, Timestamp} +import java.sql.Timestamp import org.scalatest.FunSuite @@ -34,7 +34,7 @@ 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 -> 8, TIMESTAMP -> 12, BINARY -> 16, GENERIC -> 16) + STRING -> 8, DATE -> 4, TIMESTAMP -> 12, BINARY -> 16, GENERIC -> 16) checks.foreach { case (columnType, expectedSize) => assertResult(expectedSize, s"Wrong defaultSize for $columnType") { @@ -64,7 +64,7 @@ class ColumnTypeSuite extends FunSuite with Logging { checkActualSize(FLOAT, Float.MaxValue, 4) checkActualSize(BOOLEAN, true, 1) checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) - checkActualSize(DATE, new Date(0L), 8) + checkActualSize(DATE, 0, 4) checkActualSize(TIMESTAMP, new Timestamp(0L), 12) val binary = Array.fill[Byte](4)(0: 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 f941465fa3e35..60ed28cc97bf1 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 @@ -17,11 +17,11 @@ package org.apache.spark.sql.columnar +import java.sql.Timestamp + import scala.collection.immutable.HashSet import scala.util.Random -import java.sql.{Date, Timestamp} - import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.types.{DataType, NativeType} @@ -50,7 +50,7 @@ object ColumnarTestUtils { case STRING => Random.nextString(Random.nextInt(32)) case BOOLEAN => Random.nextBoolean() case BINARY => randomBytes(Random.nextInt(32)) - case DATE => new Date(Random.nextLong()) + 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/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index cb615388da0c7..1396c6b7246d1 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 @@ -67,14 +67,15 @@ class JsonSuite extends QueryTest { checkTypePromotion(Timestamp.valueOf(strTime), enforceCorrectType(strTime, TimestampType)) val strDate = "2014-10-15" - checkTypePromotion(Date.valueOf(strDate), enforceCorrectType(strDate, DateType)) + checkTypePromotion( + DateUtils.fromJavaDate(Date.valueOf(strDate)), enforceCorrectType(strDate, DateType)) val ISO8601Time1 = "1970-01-01T01:00:01.0Z" checkTypePromotion(new Timestamp(3601000), enforceCorrectType(ISO8601Time1, TimestampType)) - checkTypePromotion(new Date(3601000), enforceCorrectType(ISO8601Time1, DateType)) + checkTypePromotion(DateUtils.millisToDays(3601000), enforceCorrectType(ISO8601Time1, DateType)) val ISO8601Time2 = "1970-01-01T02:00:01-01:00" checkTypePromotion(new Timestamp(10801000), enforceCorrectType(ISO8601Time2, TimestampType)) - checkTypePromotion(new Date(10801000), enforceCorrectType(ISO8601Time2, DateType)) + checkTypePromotion(DateUtils.millisToDays(10801000), enforceCorrectType(ISO8601Time2, DateType)) } test("Get compatible type") { 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 0d934620aca09..a6266f611c219 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 @@ -357,6 +357,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "database_drop", "database_location", "database_properties", + "date_1", "date_2", "date_3", "date_4", 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 b746942cb1067..724bd28d4b608 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive import java.io.{BufferedReader, InputStreamReader, PrintStream} -import java.sql.{Date, Timestamp} +import java.sql.Timestamp import scala.collection.JavaConversions._ import scala.language.implicitConversions @@ -409,7 +409,7 @@ private object HiveContext { toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) }.toSeq.sorted.mkString("{", ",", "}") case (null, _) => "NULL" - case (d: Date, DateType) => new DateWritable(d).toString + case (d: Int, DateType) => new DateWritable(d).toString case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8") case (decimal: java.math.BigDecimal, DecimalType()) => 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 82dba99900df9..4afa2e71d77cc 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 @@ -267,7 +267,8 @@ private[hive] trait HiveInspectors { val temp = new Array[Byte](writable.getLength) System.arraycopy(writable.getBytes, 0, temp, 0, temp.length) temp - case poi: WritableConstantDateObjectInspector => poi.getWritableConstantValue.get() + case poi: WritableConstantDateObjectInspector => + DateUtils.fromJavaDate(poi.getWritableConstantValue.get()) case mi: StandardConstantMapObjectInspector => // take the value from the map inspector object, rather than the input data mi.getWritableConstantValue.map { case (k, v) => @@ -304,7 +305,8 @@ private[hive] trait HiveInspectors { System.arraycopy(bw.getBytes(), 0, result, 0, bw.getLength()) result case x: DateObjectInspector if x.preferWritable() => - x.getPrimitiveWritableObject(data).get() + DateUtils.fromJavaDate(x.getPrimitiveWritableObject(data).get()) + case x: DateObjectInspector => DateUtils.fromJavaDate(x.getPrimitiveJavaObject(data)) // org.apache.hadoop.hive.serde2.io.TimestampWritable.set will reset current time object // if next timestamp is null, so Timestamp object is cloned case x: TimestampObjectInspector if x.preferWritable() => @@ -343,6 +345,9 @@ private[hive] trait HiveInspectors { case _: JavaHiveDecimalObjectInspector => (o: Any) => HiveShim.createDecimal(o.asInstanceOf[Decimal].toJavaBigDecimal) + case _: JavaDateObjectInspector => + (o: Any) => DateUtils.toJavaDate(o.asInstanceOf[Int]) + case soi: StandardStructObjectInspector => val wrappers = soi.getAllStructFieldRefs.map(ref => wrapperFor(ref.getFieldObjectInspector)) (o: Any) => { @@ -426,7 +431,7 @@ private[hive] trait HiveInspectors { case _: BinaryObjectInspector if x.preferWritable() => HiveShim.getBinaryWritable(a) case _: BinaryObjectInspector => a.asInstanceOf[Array[Byte]] case _: DateObjectInspector if x.preferWritable() => HiveShim.getDateWritable(a) - case _: DateObjectInspector => a.asInstanceOf[java.sql.Date] + case _: DateObjectInspector => DateUtils.toJavaDate(a.asInstanceOf[Int]) case _: TimestampObjectInspector if x.preferWritable() => HiveShim.getTimestampWritable(a) case _: TimestampObjectInspector => a.asInstanceOf[java.sql.Timestamp] } 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 c368715f7c6f5..effaa5a443512 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 @@ -34,6 +34,7 @@ import org.apache.spark.SerializableWritable 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 /** * A trait for subclasses that handle table scans. @@ -306,7 +307,7 @@ private[hive] object HadoopTableReader extends HiveInspectors { row.update(ordinal, oi.getPrimitiveJavaObject(value).clone()) case oi: DateObjectInspector => (value: Any, row: MutableRow, ordinal: Int) => - row.update(ordinal, oi.getPrimitiveJavaObject(value)) + row.update(ordinal, DateUtils.fromJavaDate(oi.getPrimitiveJavaObject(value))) case oi: BinaryObjectInspector => (value: Any, row: MutableRow, ordinal: Int) => row.update(ordinal, oi.getPrimitiveJavaObject(value)) diff --git a/sql/hive/src/test/resources/golden/Date cast-0-a7cd69b80c77a771a2c955db666be53d b/sql/hive/src/test/resources/golden/Date cast-0-a7cd69b80c77a771a2c955db666be53d new file mode 100644 index 0000000000000..98da82fa89386 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Date cast-0-a7cd69b80c77a771a2c955db666be53d @@ -0,0 +1 @@ +1970-01-01 1970-01-01 1969-12-31 16:00:00 1969-12-31 16:00:00 1970-01-01 00:00:00 diff --git a/sql/hive/src/test/resources/golden/Date comparison test 1-0-bde89be08a12361073ff658fef768b7e b/sql/hive/src/test/resources/golden/Date comparison test 1-0-bde89be08a12361073ff658fef768b7e new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Date comparison test 1-0-bde89be08a12361073ff658fef768b7e @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/Date comparison test 2-0-dc1b267f1d79d49e6675afe4fd2a34a5 b/sql/hive/src/test/resources/golden/Date comparison test 2-0-dc1b267f1d79d49e6675afe4fd2a34a5 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Date comparison test 2-0-dc1b267f1d79d49e6675afe4fd2a34a5 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/date_1-0-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/date_1-0-50131c0ba7b7a6b65c789a5a8497bada new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-0-50131c0ba7b7a6b65c789a5a8497bada @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/date_1-0-23edf29bf7376c70d5ecf12720f4b1eb b/sql/hive/src/test/resources/golden/date_1-1-23edf29bf7376c70d5ecf12720f4b1eb similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-0-23edf29bf7376c70d5ecf12720f4b1eb rename to sql/hive/src/test/resources/golden/date_1-1-23edf29bf7376c70d5ecf12720f4b1eb diff --git a/sql/hive/src/test/resources/golden/date_1-3-df16364a220ff96a6ea1cd478cbc1d0b b/sql/hive/src/test/resources/golden/date_1-10-df16364a220ff96a6ea1cd478cbc1d0b similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-3-df16364a220ff96a6ea1cd478cbc1d0b rename to sql/hive/src/test/resources/golden/date_1-10-df16364a220ff96a6ea1cd478cbc1d0b diff --git a/sql/hive/src/test/resources/golden/date_1-10-d964bec7e5632091ab5cb6f6786dbbf9 b/sql/hive/src/test/resources/golden/date_1-11-d964bec7e5632091ab5cb6f6786dbbf9 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-10-d964bec7e5632091ab5cb6f6786dbbf9 rename to sql/hive/src/test/resources/golden/date_1-11-d964bec7e5632091ab5cb6f6786dbbf9 diff --git a/sql/hive/src/test/resources/golden/date_1-11-480c5f024a28232b7857be327c992509 b/sql/hive/src/test/resources/golden/date_1-12-480c5f024a28232b7857be327c992509 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-11-480c5f024a28232b7857be327c992509 rename to sql/hive/src/test/resources/golden/date_1-12-480c5f024a28232b7857be327c992509 diff --git a/sql/hive/src/test/resources/golden/date_1-12-4c0ed7fcb75770d8790575b586bf14f4 b/sql/hive/src/test/resources/golden/date_1-13-4c0ed7fcb75770d8790575b586bf14f4 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-12-4c0ed7fcb75770d8790575b586bf14f4 rename to sql/hive/src/test/resources/golden/date_1-13-4c0ed7fcb75770d8790575b586bf14f4 diff --git a/sql/hive/src/test/resources/golden/date_1-13-44fc74c1993062c0a9522199ff27fea b/sql/hive/src/test/resources/golden/date_1-14-44fc74c1993062c0a9522199ff27fea similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-13-44fc74c1993062c0a9522199ff27fea rename to sql/hive/src/test/resources/golden/date_1-14-44fc74c1993062c0a9522199ff27fea diff --git a/sql/hive/src/test/resources/golden/date_1-14-4855a66124b16d1d0d003235995ac06b b/sql/hive/src/test/resources/golden/date_1-15-4855a66124b16d1d0d003235995ac06b similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-14-4855a66124b16d1d0d003235995ac06b rename to sql/hive/src/test/resources/golden/date_1-15-4855a66124b16d1d0d003235995ac06b diff --git a/sql/hive/src/test/resources/golden/date_1-15-8bc190dba0f641840b5e1e198a14c55b b/sql/hive/src/test/resources/golden/date_1-16-8bc190dba0f641840b5e1e198a14c55b similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-15-8bc190dba0f641840b5e1e198a14c55b rename to sql/hive/src/test/resources/golden/date_1-16-8bc190dba0f641840b5e1e198a14c55b diff --git a/sql/hive/src/test/resources/golden/date_1-1-4ebe3571c13a8b0c03096fbd972b7f1b b/sql/hive/src/test/resources/golden/date_1-17-23edf29bf7376c70d5ecf12720f4b1eb similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-1-4ebe3571c13a8b0c03096fbd972b7f1b rename to sql/hive/src/test/resources/golden/date_1-17-23edf29bf7376c70d5ecf12720f4b1eb diff --git a/sql/hive/src/test/resources/golden/date_1-16-23edf29bf7376c70d5ecf12720f4b1eb b/sql/hive/src/test/resources/golden/date_1-2-4ebe3571c13a8b0c03096fbd972b7f1b similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-16-23edf29bf7376c70d5ecf12720f4b1eb rename to sql/hive/src/test/resources/golden/date_1-2-4ebe3571c13a8b0c03096fbd972b7f1b diff --git a/sql/hive/src/test/resources/golden/date_1-2-abdce0c0d14d3fc7441b7c134b02f99a b/sql/hive/src/test/resources/golden/date_1-3-26b5c291400dfde455b3c1b878b71d0 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-2-abdce0c0d14d3fc7441b7c134b02f99a rename to sql/hive/src/test/resources/golden/date_1-3-26b5c291400dfde455b3c1b878b71d0 diff --git a/sql/hive/src/test/resources/golden/date_1-6-df16364a220ff96a6ea1cd478cbc1d0b b/sql/hive/src/test/resources/golden/date_1-4-df16364a220ff96a6ea1cd478cbc1d0b similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-6-df16364a220ff96a6ea1cd478cbc1d0b rename to sql/hive/src/test/resources/golden/date_1-4-df16364a220ff96a6ea1cd478cbc1d0b diff --git a/sql/hive/src/test/resources/golden/date_1-4-d964bec7e5632091ab5cb6f6786dbbf9 b/sql/hive/src/test/resources/golden/date_1-5-d964bec7e5632091ab5cb6f6786dbbf9 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-4-d964bec7e5632091ab5cb6f6786dbbf9 rename to sql/hive/src/test/resources/golden/date_1-5-d964bec7e5632091ab5cb6f6786dbbf9 diff --git a/sql/hive/src/test/resources/golden/date_1-5-5e70fc74158fbfca38134174360de12d b/sql/hive/src/test/resources/golden/date_1-6-559d01fb0b42c42f0c4927fa0f9deac4 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-5-5e70fc74158fbfca38134174360de12d rename to sql/hive/src/test/resources/golden/date_1-6-559d01fb0b42c42f0c4927fa0f9deac4 diff --git a/sql/hive/src/test/resources/golden/date_1-9-df16364a220ff96a6ea1cd478cbc1d0b b/sql/hive/src/test/resources/golden/date_1-7-df16364a220ff96a6ea1cd478cbc1d0b similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-9-df16364a220ff96a6ea1cd478cbc1d0b rename to sql/hive/src/test/resources/golden/date_1-7-df16364a220ff96a6ea1cd478cbc1d0b diff --git a/sql/hive/src/test/resources/golden/date_1-7-d964bec7e5632091ab5cb6f6786dbbf9 b/sql/hive/src/test/resources/golden/date_1-8-d964bec7e5632091ab5cb6f6786dbbf9 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-7-d964bec7e5632091ab5cb6f6786dbbf9 rename to sql/hive/src/test/resources/golden/date_1-8-d964bec7e5632091ab5cb6f6786dbbf9 diff --git a/sql/hive/src/test/resources/golden/date_1-8-1d5c58095cd52ea539d869f2ab1ab67d b/sql/hive/src/test/resources/golden/date_1-9-8306558e0eabe936ac33dabaaa17fea4 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-8-1d5c58095cd52ea539d869f2ab1ab67d rename to sql/hive/src/test/resources/golden/date_1-9-8306558e0eabe936ac33dabaaa17fea4 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 2d3ff680125ad..09bbd5c867e4e 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 @@ -18,7 +18,6 @@ package org.apache.spark.sql.hive import java.util -import java.sql.Date import java.util.{Locale, TimeZone} import org.apache.hadoop.hive.ql.udf.UDAFPercentile @@ -76,7 +75,7 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { Literal(0.asInstanceOf[Float]) :: Literal(0.asInstanceOf[Double]) :: Literal("0") :: - Literal(new Date(2014, 9, 23)) :: + Literal(new java.sql.Date(114, 8, 23)) :: Literal(Decimal(BigDecimal(123.123))) :: Literal(new java.sql.Timestamp(123123)) :: Literal(Array[Byte](1,2,3)) :: @@ -143,7 +142,6 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { 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) } - case (r1: Date, r2: Date) => assert(r1.compareTo(r2) === 0) case (r1, r2) => assert(r1 === r2) } } 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 4c53b10ba96e9..4f67d1def65fc 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 @@ -253,8 +253,30 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("Cast Timestamp to Timestamp in UDF", """ - | SELECT DATEDIFF(CAST(value AS timestamp), CAST('2002-03-21 00:00:00' AS timestamp)) - | FROM src LIMIT 1 + | SELECT DATEDIFF(CAST(value AS timestamp), CAST('2002-03-21 00:00:00' AS timestamp)) + | FROM src LIMIT 1 + """.stripMargin) + + createQueryTest("Date comparison test 1", + """ + | SELECT + | CAST(CAST('1970-01-01 22:00:00' AS timestamp) AS date) == + | CAST(CAST('1970-01-01 23:00:00' AS timestamp) AS date) + | FROM src LIMIT 1 + """.stripMargin) + + createQueryTest("Date comparison test 2", + "SELECT CAST(CAST(0 AS timestamp) AS date) > CAST(0 AS timestamp) FROM src LIMIT 1") + + createQueryTest("Date cast", + """ + | SELECT + | CAST(CAST(0 AS timestamp) AS date), + | CAST(CAST(CAST(0 AS timestamp) AS date) AS string), + | CAST(0 AS timestamp), + | CAST(CAST(0 AS timestamp) AS string), + | CAST(CAST(CAST('1970-01-01 23:00:00' AS timestamp) AS date) AS timestamp) + | FROM src LIMIT 1 """.stripMargin) createQueryTest("Simple Average", 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 254919e8f6fdc..b5a0754ff61f9 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 @@ -160,7 +160,7 @@ private[hive] object HiveShim { if (value == null) null else new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]]) def getDateWritable(value: Any): hiveIo.DateWritable = - if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[java.sql.Date]) + if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[Int]) def getTimestampWritable(value: Any): hiveIo.TimestampWritable = if (value == null) { 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 45ca59ae56a38..e4c1809c8bb21 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 @@ -263,7 +263,7 @@ private[hive] object HiveShim { } def getDateWritable(value: Any): hiveIo.DateWritable = - if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[java.sql.Date]) + if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[Int]) def getTimestampWritable(value: Any): hiveIo.TimestampWritable = if (value == null) { From 46d50f151c02c6892fc84a37fdf2a521dc774d1c Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 2 Feb 2015 15:55:44 -0800 Subject: [PATCH 015/578] [SPARK-5513][MLLIB] Add nonnegative option to ml's ALS This PR ports the NNLS solver to the new ALS implementation. CC: coderxiang Author: Xiangrui Meng Closes #4302 from mengxr/SPARK-5513 and squashes the following commits: 4cbdab0 [Xiangrui Meng] fix serialization 88de634 [Xiangrui Meng] add NNLS to ml's ALS --- .../apache/spark/ml/recommendation/ALS.scala | 95 ++++++++++++++++--- .../spark/mllib/optimization/NNLS.scala | 4 +- .../spark/ml/recommendation/ALSSuite.scala | 11 +++ 3 files changed, 96 insertions(+), 14 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 979a19d3b2057..82d21d5e4cb6e 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 @@ -25,12 +25,14 @@ import scala.util.Sorting 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.{HashPartitioner, 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.mllib.optimization.NNLS import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame import org.apache.spark.sql.Dsl._ @@ -80,6 +82,10 @@ private[recommendation] trait ALSParams extends Params with HasMaxIter with HasR val ratingCol = new Param[String](this, "ratingCol", "column name for ratings", Some("rating")) def getRatingCol: String = get(ratingCol) + val nonnegative = new BooleanParam( + this, "nonnegative", "whether to use nonnegative constraint for least squares", Some(false)) + val getNonnegative: Boolean = get(nonnegative) + /** * Validates and transforms the input schema. * @param schema input schema @@ -186,6 +192,7 @@ class ALS extends Estimator[ALSModel] with ALSParams { def setPredictionCol(value: String): this.type = set(predictionCol, value) def setMaxIter(value: Int): this.type = set(maxIter, value) def setRegParam(value: Double): this.type = set(regParam, value) + def setNonnegative(value: Boolean): this.type = set(nonnegative, value) /** Sets both numUserBlocks and numItemBlocks to the specific value. */ def setNumBlocks(value: Int): this.type = { @@ -207,7 +214,7 @@ 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)) + alpha = map(alpha), nonnegative = map(nonnegative)) val model = new ALSModel(this, map, map(rank), userFactors, itemFactors) Params.inheritValues(map, this, model) model @@ -232,11 +239,16 @@ object ALS extends Logging { /** Rating class for better code readability. */ case class Rating[@specialized(Int, Long) ID](user: ID, item: ID, rating: Float) + /** 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). */ + def solve(ne: NormalEquation, lambda: Double): Array[Float] + } + /** Cholesky solver for least square problems. */ - private[recommendation] class CholeskySolver { + private[recommendation] class CholeskySolver extends LeastSquaresNESolver { private val upper = "U" - private val info = new intW(0) /** * Solves a least squares problem with L2 regularization: @@ -247,7 +259,7 @@ object ALS extends Logging { * @param lambda regularization constant, which will be scaled by n * @return the solution x */ - def solve(ne: NormalEquation, lambda: Double): Array[Float] = { + 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 @@ -258,6 +270,7 @@ object ALS extends Logging { i += j j += 1 } + val info = new intW(0) lapack.dppsv(upper, k, 1, ne.ata, ne.atb, k, info) val code = info.`val` assert(code == 0, s"lapack.dppsv returned $code.") @@ -272,6 +285,63 @@ object ALS extends Logging { } } + /** NNLS solver. */ + private[recommendation] class NNLSSolver extends LeastSquaresNESolver { + private var rank: Int = -1 + private var workspace: NNLS.Workspace = _ + private var ata: DoubleMatrix = _ + 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) + initialized = true + } else { + require(this.rank == rank) + } + } + + /** + * Solves a nonnegative least squares problem with L2 regularizatin: + * + * min_x_ norm(A x - b)^2^ + lambda * n * norm(x)^2^ + * subject to x >= 0 + */ + override def solve(ne: NormalEquation, lambda: Double): Array[Float] = { + val rank = ne.k + initialize(rank) + fillAtA(ne.ata, lambda * ne.n) + val x = NNLS.solve(ata, new DoubleMatrix(rank, 1, ne.atb: _*), workspace) + ne.reset() + x.map(x => x.toFloat) + } + + /** + * Given a triangular matrix in the order of fillXtX above, compute the full symmetric square + * matrix that it represents, storing it into destMatrix. + */ + private def fillAtA(triAtA: Array[Double], lambda: Double) { + 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 + pos += 1 + j += 1 + } + data(i * rank + i) += lambda + i += 1 + } + } + } + /** Representing a normal equation (ALS' subproblem). */ private[recommendation] class NormalEquation(val k: Int) extends Serializable { @@ -350,12 +420,14 @@ object ALS extends Logging { maxIter: Int = 10, regParam: Double = 1.0, implicitPrefs: Boolean = false, - alpha: Double = 1.0)( + alpha: Double = 1.0, + nonnegative: Boolean = false)( implicit ord: Ordering[ID]): (RDD[(ID, Array[Float])], RDD[(ID, Array[Float])]) = { val userPart = new HashPartitioner(numUserBlocks) val itemPart = new HashPartitioner(numItemBlocks) val userLocalIndexEncoder = new LocalIndexEncoder(userPart.numPartitions) val itemLocalIndexEncoder = new LocalIndexEncoder(itemPart.numPartitions) + val solver = if (nonnegative) new NNLSSolver else new CholeskySolver val blockRatings = partitionRatings(ratings, userPart, itemPart).cache() val (userInBlocks, userOutBlocks) = makeBlocks("user", blockRatings, userPart, itemPart) // materialize blockRatings and user blocks @@ -374,20 +446,20 @@ object ALS extends Logging { userFactors.setName(s"userFactors-$iter").persist() val previousItemFactors = itemFactors itemFactors = computeFactors(userFactors, userOutBlocks, itemInBlocks, rank, regParam, - userLocalIndexEncoder, implicitPrefs, alpha) + userLocalIndexEncoder, implicitPrefs, alpha, solver) previousItemFactors.unpersist() itemFactors.setName(s"itemFactors-$iter").persist() val previousUserFactors = userFactors userFactors = computeFactors(itemFactors, itemOutBlocks, userInBlocks, rank, regParam, - itemLocalIndexEncoder, implicitPrefs, alpha) + itemLocalIndexEncoder, implicitPrefs, alpha, solver) previousUserFactors.unpersist() } } else { for (iter <- 0 until maxIter) { itemFactors = computeFactors(userFactors, userOutBlocks, itemInBlocks, rank, regParam, - userLocalIndexEncoder) + userLocalIndexEncoder, solver = solver) userFactors = computeFactors(itemFactors, itemOutBlocks, userInBlocks, rank, regParam, - itemLocalIndexEncoder) + itemLocalIndexEncoder, solver = solver) } } val userIdAndFactors = userInBlocks @@ -879,6 +951,7 @@ object ALS extends Logging { * @param srcEncoder encoder for src local indices * @param implicitPrefs whether to use implicit preference * @param alpha the alpha constant in the implicit preference formulation + * @param solver solver for least squares problems * * @return dst factors */ @@ -890,7 +963,8 @@ object ALS extends Logging { regParam: Double, srcEncoder: LocalIndexEncoder, implicitPrefs: Boolean = false, - alpha: Double = 1.0): RDD[(Int, FactorBlock)] = { + alpha: Double = 1.0, + solver: LeastSquaresNESolver): RDD[(Int, FactorBlock)] = { val numSrcBlocks = srcFactorBlocks.partitions.length val YtY = if (implicitPrefs) Some(computeYtY(srcFactorBlocks, rank)) else None val srcOut = srcOutBlocks.join(srcFactorBlocks).flatMap { @@ -909,7 +983,6 @@ object ALS extends Logging { val dstFactors = new Array[Array[Float]](dstIds.length) var j = 0 val ls = new NormalEquation(rank) - val solver = new CholeskySolver // TODO: add NNLS solver while (j < dstIds.length) { ls.reset() if (implicitPrefs) { 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 fef062e02b6ec..ccd93b318bc23 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 @@ -19,13 +19,11 @@ package org.apache.spark.mllib.optimization import org.jblas.{DoubleMatrix, SimpleBlas} -import org.apache.spark.annotation.DeveloperApi - /** * Object used to solve nonnegative least squares problems using a modified * projected gradient method. */ -private[mllib] object NNLS { +private[spark] object NNLS { class Workspace(val n: Int) { val scratch = new DoubleMatrix(n, 1) val grad = new DoubleMatrix(n, 1) 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 07aff56fb7d2f..ee08c3c32760e 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 @@ -444,4 +444,15 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext with Logging { val (strUserFactors, _) = ALS.train(strRatings, rank = 2, maxIter = 4) assert(strUserFactors.first()._1.getClass === classOf[String]) } + + test("nonnegative constraint") { + val (ratings, _) = genImplicitTestData(numUsers = 20, numItems = 40, rank = 2, noiseStd = 0.01) + val (userFactors, itemFactors) = ALS.train(ratings, rank = 2, maxIter = 4, nonnegative = true) + def isNonnegative(factors: RDD[(Int, Array[Float])]): Boolean = { + factors.values.map { _.forall(_ >= 0.0) }.reduce(_ && _) + } + assert(isNonnegative(userFactors)) + assert(isNonnegative(itemFactors)) + // TODO: Validate the solution. + } } From b1aa8fe988301b924048039529234278aeb0298a Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Mon, 2 Feb 2015 15:59:15 -0800 Subject: [PATCH 016/578] [SPARK-2309][MLlib] Multinomial Logistic Regression #1379 is automatically closed by asfgit, and github can not reopen it once it's closed, so this will be the new PR. Binary Logistic Regression can be extended to Multinomial Logistic Regression by running K-1 independent Binary Logistic Regression models. The following formula is implemented. http://www.slideshare.net/dbtsai/2014-0620-mlor-36132297/25 Author: DB Tsai Closes #3833 from dbtsai/mlor and squashes the following commits: 4e2f354 [DB Tsai] triger jenkins 697b7c9 [DB Tsai] address some feedback 4ce4d33 [DB Tsai] refactoring ff843b3 [DB Tsai] rebase f114135 [DB Tsai] refactoring 4348426 [DB Tsai] Addressed feedback from Sean Owen a252197 [DB Tsai] first commit --- .../classification/LogisticRegression.scala | 128 +++++++++-- .../spark/mllib/optimization/Gradient.scala | 200 +++++++++++++++--- .../GeneralizedLinearAlgorithm.scala | 101 +++++++-- .../spark/mllib/util/DataValidators.scala | 18 +- .../LogisticRegressionSuite.scala | 179 +++++++++++++++- 5 files changed, 565 insertions(+), 61 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 94d757bc317ab..282fb3ff283f4 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 @@ -18,30 +18,41 @@ package org.apache.spark.mllib.classification import org.apache.spark.annotation.Experimental -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.BLAS.dot +import org.apache.spark.mllib.linalg.{DenseVector, Vector} import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.DataValidators +import org.apache.spark.mllib.util.{DataValidators, MLUtils} import org.apache.spark.rdd.RDD /** - * Classification model trained using Logistic Regression. + * Classification model trained using Multinomial/Binary Logistic Regression. * * @param weights Weights computed for every feature. - * @param intercept Intercept computed for this model. + * @param intercept Intercept computed for this model. (Only used in Binary Logistic Regression. + * In Multinomial Logistic Regression, the intercepts will not be a single values, + * so the intercepts will be part of the weights.) + * @param numFeatures the dimension of the features. + * @param numClasses the number of possible outcomes for k classes classification problem in + * Multinomial Logistic Regression. By default, it is binary logistic regression + * so numClasses will be set to 2. */ class LogisticRegressionModel ( override val weights: Vector, - override val intercept: Double) + override val intercept: Double, + val numFeatures: Int, + val numClasses: Int) extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable { + def this(weights: Vector, intercept: Double) = this(weights, intercept, weights.size, 2) + private var threshold: Option[Double] = Some(0.5) /** * :: Experimental :: - * Sets the threshold that separates positive predictions from negative predictions. An example - * with prediction score greater than or equal to this threshold is identified as an positive, - * and negative otherwise. The default value is 0.5. + * 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. */ @Experimental def setThreshold(threshold: Double): this.type = { @@ -61,20 +72,68 @@ class LogisticRegressionModel ( override protected def predictPoint(dataMatrix: Vector, weightMatrix: Vector, intercept: Double) = { - val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept - val score = 1.0 / (1.0 + math.exp(-margin)) - threshold match { - case Some(t) => if (score > t) 1.0 else 0.0 - case None => score + require(dataMatrix.size == numFeatures) + + // If dataMatrix and weightMatrix have the same dimension, it's binary logistic regression. + if (numClasses == 2) { + require(numFeatures == weightMatrix.size) + val margin = dot(weights, dataMatrix) + intercept + val score = 1.0 / (1.0 + math.exp(-margin)) + threshold match { + case Some(t) => if (score > t) 1.0 else 0.0 + case None => score + } + } else { + val dataWithBiasSize = weightMatrix.size / (numClasses - 1) + + val weightsArray = weights match { + case dv: DenseVector => dv.values + case _ => + throw new IllegalArgumentException( + s"weights only supports dense vector but got type ${weights.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. + * + * 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 + * is positive to prevent overflow. + */ + var bestClass = 0 + var maxMargin = 0.0 + var i = 0 + while(i < margins.size) { + if (margins(i) > maxMargin) { + maxMargin = margins(i) + bestClass = i + 1 + } + i += 1 + } + bestClass.toDouble } } } /** - * Train a classification model for Logistic Regression using Stochastic Gradient Descent. By - * default L2 regularization is used, which can be changed via - * [[LogisticRegressionWithSGD.optimizer]]. - * NOTE: Labels used in Logistic Regression should be {0, 1}. + * Train a classification model for Binary Logistic Regression + * using Stochastic Gradient Descent. By default L2 regularization is used, + * which can be changed via [[LogisticRegressionWithSGD.optimizer]]. + * NOTE: Labels used in Logistic Regression should be {0, 1, ..., k - 1} + * for k classes multi-label classification problem. * Using [[LogisticRegressionWithLBFGS]] is recommended over this. */ class LogisticRegressionWithSGD private ( @@ -194,9 +253,10 @@ object LogisticRegressionWithSGD { } /** - * Train a classification model for Logistic Regression using Limited-memory BFGS. - * Standard feature scaling and L2 regularization are used by default. - * NOTE: Labels used in Logistic Regression should be {0, 1} + * Train a classification model for Multinomial/Binary Logistic Regression using + * Limited-memory BFGS. Standard feature scaling and L2 regularization are used by default. + * NOTE: Labels used in Logistic Regression should be {0, 1, ..., k - 1} + * for k classes multi-label classification problem. */ class LogisticRegressionWithLBFGS extends GeneralizedLinearAlgorithm[LogisticRegressionModel] with Serializable { @@ -205,9 +265,33 @@ class LogisticRegressionWithLBFGS override val optimizer = new LBFGS(new LogisticGradient, new SquaredL2Updater) - override protected val validators = List(DataValidators.binaryLabelValidator) + override protected val validators = List(multiLabelValidator) + + private def multiLabelValidator: RDD[LabeledPoint] => Boolean = { data => + if (numOfLinearPredictor > 1) { + DataValidators.multiLabelValidator(numOfLinearPredictor + 1)(data) + } else { + DataValidators.binaryLabelValidator(data) + } + } + + /** + * :: Experimental :: + * Set the number of possible outcomes for k classes classification problem in + * Multinomial Logistic Regression. + * By default, it is binary logistic regression so k will be set to 2. + */ + @Experimental + def setNumClasses(numClasses: Int): this.type = { + require(numClasses > 1) + numOfLinearPredictor = numClasses - 1 + if (numClasses > 2) { + optimizer.setGradient(new LogisticGradient(numClasses)) + } + this + } override protected def createModel(weights: Vector, intercept: Double) = { - new LogisticRegressionModel(weights, intercept) + new LogisticRegressionModel(weights, intercept, numFeatures, numOfLinearPredictor + 1) } } 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 1ca0f36c6ac34..0acdab797e8f3 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 @@ -18,7 +18,7 @@ package org.apache.spark.mllib.optimization import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{DenseVector, Vector, Vectors} import org.apache.spark.mllib.linalg.BLAS.{axpy, dot, scal} import org.apache.spark.mllib.util.MLUtils @@ -55,24 +55,86 @@ abstract class Gradient extends Serializable { /** * :: DeveloperApi :: - * Compute gradient and loss for a logistic loss function, as used in binary classification. - * See also the documentation for the precise formulation. + * Compute gradient and loss for a multinomial logistic loss function, as used + * in multi-class classification (it is also used in binary logistic regression). + * + * In `The Elements of Statistical Learning: Data Mining, Inference, and Prediction, 2nd Edition` + * by Trevor Hastie, Robert Tibshirani, and Jerome Friedman, which can be downloaded from + * 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. + * + * The model weights w = (w_1, w_2, ..., w_{K-1})^T becomes a matrix which has dimension of + * (K-1) * (N+1) if the intercepts are added. If the intercepts are not added, the dimension + * 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, + * margins_i = x w_i. + * + * 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}) + * + * 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 + * in data which are far away from hyperplane, and this will cause the failing of training once + * infinity / infinity is introduced. Note that this is only a concern when max(margins) > 0. + * + * 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. + * + * Note that each term, (margins_i - maxMargin) in \exp is smaller than zero; as a result, + * overflow will not happen with this formula. + * + * 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. + * + * For the detailed mathematical derivation, see the reference at + * http://www.slideshare.net/dbtsai/2014-0620-mlor-36132297 + * + * @param numClasses the number of possible outcomes for k classes classification problem in + * Multinomial Logistic Regression. By default, it is binary logistic regression + * so numClasses will be set to 2. */ @DeveloperApi -class LogisticGradient extends Gradient { - override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { - val margin = -1.0 * dot(data, weights) - val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label - val gradient = data.copy - scal(gradientMultiplier, gradient) - val loss = - if (label > 0) { - // The following is equivalent to log(1 + exp(margin)) but more numerically stable. - MLUtils.log1pExp(margin) - } else { - MLUtils.log1pExp(margin) - margin - } +class LogisticGradient(numClasses: Int) extends Gradient { + def this() = this(2) + + override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { + val gradient = Vectors.zeros(weights.size) + val loss = compute(data, label, weights, gradient) (gradient, loss) } @@ -81,14 +143,104 @@ class LogisticGradient extends Gradient { label: Double, weights: Vector, cumGradient: Vector): Double = { - val margin = -1.0 * dot(data, weights) - val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label - axpy(gradientMultiplier, data, cumGradient) - if (label > 0) { - // The following is equivalent to log(1 + exp(margin)) but more numerically stable. - MLUtils.log1pExp(margin) - } else { - MLUtils.log1pExp(margin) - margin + val dataSize = data.size + + // (weights.size / dataSize + 1) is number of classes + require(weights.size % dataSize == 0 && numClasses == weights.size / dataSize + 1) + numClasses match { + case 2 => + /** + * For Binary Logistic Regression. + * + * Although the loss and gradient calculation for multinomial one is more generalized, + * and multinomial one can also be used in binary case, we still implement a specialized + * binary version for performance reason. + */ + val margin = -1.0 * dot(data, weights) + val multiplier = (1.0 / (1.0 + math.exp(margin))) - label + axpy(multiplier, data, cumGradient) + if (label > 0) { + // The following is equivalent to log(1 + exp(margin)) but more numerically stable. + MLUtils.log1pExp(margin) + } else { + MLUtils.log1pExp(margin) - margin + } + case _ => + /** + * For Multinomial Logistic Regression. + */ + val weightsArray = weights match { + case dv: DenseVector => dv.values + case _ => + throw new IllegalArgumentException( + s"weights only supports dense vector but got type ${weights.getClass}.") + } + val cumGradientArray = cumGradient match { + case dv: DenseVector => dv.values + case _ => + throw new IllegalArgumentException( + s"cumGradient only supports dense vector but got type ${cumGradient.getClass}.") + } + + // marginY is margins(label - 1) in the formula. + var marginY = 0.0 + var maxMargin = Double.NegativeInfinity + var maxMarginIndex = 0 + + val margins = Array.tabulate(numClasses - 1) { i => + var margin = 0.0 + data.foreachActive { (index, value) => + if (value != 0.0) margin += value * weightsArray((i * dataSize) + index) + } + if (i == label.toInt - 1) marginY = margin + if (margin > maxMargin) { + maxMargin = margin + maxMarginIndex = i + } + margin + } + + /** + * When maxMargin > 0, the original formula will cause overflow as we discuss + * in the previous comment. + * We address this by subtracting maxMargin from all the margins, so it's guaranteed + * that all of the new margins will be smaller than zero to prevent arithmetic overflow. + */ + val sum = { + var temp = 0.0 + if (maxMargin > 0) { + for (i <- 0 until numClasses - 1) { + margins(i) -= maxMargin + if (i == maxMarginIndex) { + temp += math.exp(-maxMargin) + } else { + temp += math.exp(margins(i)) + } + } + } else { + for (i <- 0 until numClasses - 1) { + temp += math.exp(margins(i)) + } + } + temp + } + + for (i <- 0 until numClasses - 1) { + val multiplier = math.exp(margins(i)) / (sum + 1.0) - { + if (label != 0.0 && label == i + 1) 1.0 else 0.0 + } + data.foreachActive { (index, value) => + if (value != 0.0) cumGradientArray(i * dataSize + index) += multiplier * value + } + } + + val loss = if (label > 0.0) math.log1p(sum) - marginY else math.log1p(sum) + + if (maxMargin > 0) { + loss + maxMargin + } else { + loss + } } } } 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 0287f04e2c777..17de215b97f9d 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 @@ -98,6 +98,23 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] protected var validateData: Boolean = true + /** + * In `GeneralizedLinearModel`, only single linear predictor is allowed for both weights + * and intercept. However, for multinomial logistic regression, with K possible outcomes, + * we are training K-1 independent binary logistic regression models which requires K-1 sets + * of linear predictor. + * + * As a result, the workaround here is if more than two sets of linear predictors are needed, + * we construct bigger `weights` vector which can hold both weights and intercepts. + * If the intercepts are added, the dimension of `weights` will be + * (numOfLinearPredictor) * (numFeatures + 1) . If the intercepts are not added, + * the dimension of `weights` will be (numOfLinearPredictor) * numFeatures. + * + * Thus, the intercepts will be encapsulated into weights, and we leave the value of intercept + * in GeneralizedLinearModel as zero. + */ + protected var numOfLinearPredictor: Int = 1 + /** * Whether to perform feature scaling before model training to reduce the condition numbers * which can significantly help the optimizer converging faster. The scaling correction will be @@ -106,6 +123,11 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] */ private var useFeatureScaling = false + /** + * The dimension of training features. + */ + protected var numFeatures: Int = 0 + /** * Set if the algorithm should use feature scaling to improve the convergence during optimization. */ @@ -141,8 +163,28 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] * RDD of LabeledPoint entries. */ def run(input: RDD[LabeledPoint]): M = { - val numFeatures: Int = input.first().features.size - val initialWeights = Vectors.dense(new Array[Double](numFeatures)) + numFeatures = input.first().features.size + + /** + * When `numOfLinearPredictor > 1`, the intercepts are encapsulated into weights, + * so the `weights` will include the intercepts. When `numOfLinearPredictor == 1`, + * the intercept will be stored as separated value in `GeneralizedLinearModel`. + * This will result in different behaviors since when `numOfLinearPredictor == 1`, + * users have no way to set the initial intercept, while in the other case, users + * can set the intercepts as part of weights. + * + * TODO: See if we can deprecate `intercept` in `GeneralizedLinearModel`, and always + * have the intercept as part of weights to have consistent design. + */ + val initialWeights = { + if (numOfLinearPredictor == 1) { + Vectors.dense(new Array[Double](numFeatures)) + } else if (addIntercept) { + Vectors.dense(new Array[Double]((numFeatures + 1) * numOfLinearPredictor)) + } else { + Vectors.dense(new Array[Double](numFeatures * numOfLinearPredictor)) + } + } run(input, initialWeights) } @@ -151,6 +193,7 @@ 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" @@ -182,14 +225,14 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] * Currently, it's only enabled in LogisticRegressionWithLBFGS */ val scaler = if (useFeatureScaling) { - (new StandardScaler).fit(input.map(x => x.features)) + (new StandardScaler(withStd = true, withMean = false)).fit(input.map(x => x.features)) } else { null } // Prepend an extra variable consisting of all 1.0's for the intercept. val data = if (addIntercept) { - if(useFeatureScaling) { + if (useFeatureScaling) { input.map(labeledPoint => (labeledPoint.label, appendBias(scaler.transform(labeledPoint.features)))) } else { @@ -203,21 +246,31 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] } } - val initialWeightsWithIntercept = if (addIntercept) { + /** + * TODO: For better convergence, in logistic regression, the intercepts should be computed + * from the prior probability distribution of the outcomes; for linear regression, + * the intercept should be set as the average of response. + */ + val initialWeightsWithIntercept = if (addIntercept && numOfLinearPredictor == 1) { appendBias(initialWeights) } else { + /** If `numOfLinearPredictor > 1`, initialWeights already contains intercepts. */ initialWeights } val weightsWithIntercept = optimizer.optimize(data, initialWeightsWithIntercept) - val intercept = if (addIntercept) weightsWithIntercept(weightsWithIntercept.size - 1) else 0.0 - var weights = - if (addIntercept) { - Vectors.dense(weightsWithIntercept.toArray.slice(0, weightsWithIntercept.size - 1)) - } else { - weightsWithIntercept - } + val intercept = if (addIntercept && numOfLinearPredictor == 1) { + weightsWithIntercept(weightsWithIntercept.size - 1) + } else { + 0.0 + } + + var weights = if (addIntercept && numOfLinearPredictor == 1) { + Vectors.dense(weightsWithIntercept.toArray.slice(0, weightsWithIntercept.size - 1)) + } else { + weightsWithIntercept + } /** * The weights and intercept are trained in the scaled space; we're converting them back to @@ -228,7 +281,29 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] * is the coefficient in the original space, and v_i is the variance of the column i. */ if (useFeatureScaling) { - weights = scaler.transform(weights) + if (numOfLinearPredictor == 1) { + weights = scaler.transform(weights) + } else { + /** + * For `numOfLinearPredictor > 1`, we have to transform the weights back to the original + * scale for each set of linear predictor. Note that the intercepts have to be explicitly + * excluded when `addIntercept == true` since the intercepts are part of weights now. + */ + var i = 0 + val n = weights.size / numOfLinearPredictor + val weightsArray = weights.toArray + while (i < numOfLinearPredictor) { + val start = i * n + val end = (i + 1) * n - { if (addIntercept) 1 else 0 } + + val partialWeightsArray = scaler.transform( + Vectors.dense(weightsArray.slice(start, end))).toArray + + System.arraycopy(partialWeightsArray, 0, weightsArray, start, partialWeightsArray.size) + i += 1 + } + weights = Vectors.dense(weightsArray) + } } // Warn at the end of the run as well, for increased visibility. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala index 45f95482a1def..be335a1aca58a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala @@ -34,11 +34,27 @@ object DataValidators extends Logging { * * @return True if labels are all zero or one, false otherwise. */ - val binaryLabelValidator: RDD[LabeledPoint] => Boolean = { data => + val binaryLabelValidator: RDD[LabeledPoint] => Boolean = { data => val numInvalid = data.filter(x => x.label != 1.0 && x.label != 0.0).count() if (numInvalid != 0) { logError("Classification labels should be 0 or 1. Found " + numInvalid + " invalid labels") } numInvalid == 0 } + + /** + * Function to check if labels used for k class multi-label classification are + * in the range of {0, 1, ..., k - 1}. + * + * @return True if labels are all in the range of {0, 1, ..., k-1}, false otherwise. + */ + def multiLabelValidator(k: Int): RDD[LabeledPoint] => Boolean = { data => + val numInvalid = data.filter(x => + x.label - x.label.toInt != 0.0 || x.label < 0 || x.label > k - 1).count() + if (numInvalid != 0) { + logError("Classification labels should be in {0 to " + (k - 1) + "}. " + + "Found " + numInvalid + " invalid labels") + } + numInvalid == 0 + } } 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 94b0e00f37267..3fb45938f75db 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 @@ -17,13 +17,14 @@ package org.apache.spark.mllib.classification +import scala.util.control.Breaks._ import scala.util.Random import scala.collection.JavaConversions._ import org.scalatest.FunSuite import org.scalatest.Matchers -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.{LocalClusterSparkContext, MLlibTestSparkContext} import org.apache.spark.mllib.util.TestingUtils._ @@ -55,6 +56,97 @@ object LogisticRegressionSuite { val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Vectors.dense(Array(x1(i))))) testData } + + /** + * Generates `k` classes multinomial synthetic logistic input in `n` dimensional space given the + * model weights and mean/variance of the features. The synthetic data will be drawn from + * the probability distribution constructed by weights using the following formula. + * + * P(y = 0 | x) = 1 / norm + * P(y = 1 | x) = exp(x * w_1) / norm + * P(y = 2 | x) = exp(x * w_2) / norm + * ... + * P(y = k-1 | x) = exp(x * w_{k-1}) / norm + * where norm = 1 + exp(x * w_1) + exp(x * w_2) + ... + exp(x * w_{k-1}) + * + * @param weights matrix is flatten into a vector; as a result, the dimension of weights vector + * will be (k - 1) * (n + 1) if `addIntercept == true`, and + * if `addIntercept != true`, the dimension will be (k - 1) * n. + * @param xMean the mean of the generated features. Lots of time, if the features are not properly + * standardized, the algorithm with poor implementation will have difficulty + * to converge. + * @param xVariance the variance of the generated features. + * @param addIntercept whether to add intercept. + * @param nPoints the number of instance of generated data. + * @param seed the seed for random generator. For consistent testing result, it will be fixed. + */ + def generateMultinomialLogisticInput( + weights: Array[Double], + xMean: Array[Double], + xVariance: Array[Double], + addIntercept: Boolean, + nPoints: Int, + seed: Int): Seq[LabeledPoint] = { + val rnd = new Random(seed) + + val xDim = xMean.size + val xWithInterceptsDim = if (addIntercept) xDim + 1 else xDim + val nClasses = weights.size / xWithInterceptsDim + 1 + + val x = Array.fill[Vector](nPoints)(Vectors.dense(Array.fill[Double](xDim)(rnd.nextGaussian()))) + + x.map(vector => { + // This doesn't work if `vector` is a sparse vector. + val vectorArray = vector.toArray + var i = 0 + while (i < vectorArray.size) { + vectorArray(i) = vectorArray(i) * math.sqrt(xVariance(i)) + xMean(i) + i += 1 + } + }) + + val y = (0 until nPoints).map { idx => + val xArray = x(idx).toArray + val margins = Array.ofDim[Double](nClasses) + val probs = Array.ofDim[Double](nClasses) + + for (i <- 0 until nClasses - 1) { + for (j <- 0 until xDim) margins(i + 1) += weights(i * xWithInterceptsDim + j) * xArray(j) + if (addIntercept) margins(i + 1) += weights((i + 1) * xWithInterceptsDim - 1) + } + // Preventing the overflow when we compute the probability + val maxMargin = margins.max + if (maxMargin > 0) for (i <-0 until nClasses) margins(i) -= maxMargin + + // Computing the probabilities for each class from the margins. + val norm = { + var temp = 0.0 + for (i <- 0 until nClasses) { + probs(i) = math.exp(margins(i)) + temp += probs(i) + } + temp + } + for (i <-0 until nClasses) probs(i) /= norm + + // Compute the cumulative probability so we can generate a random number and assign a label. + for (i <- 1 until nClasses) probs(i) += probs(i - 1) + val p = rnd.nextDouble() + var y = 0 + breakable { + for (i <- 0 until nClasses) { + if (p < probs(i)) { + y = i + break + } + } + } + y + } + + val testData = (0 until nPoints).map(i => LabeledPoint(y(i), x(i))) + testData + } } class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext with Matchers { @@ -285,6 +377,91 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext with M assert(modelB1.weights(0) !~== modelB3.weights(0) * 1.0E6 absTol 0.1) } + test("multinomial logistic regression with LBFGS") { + val nPoints = 10000 + + /** + * The following weights and xMean/xVariance are computed from iris dataset with lambda = 0.2. + * As a result, we are actually drawing samples from probability distribution of built model. + */ + val weights = Array( + -0.57997, 0.912083, -0.371077, -0.819866, 2.688191, + -0.16624, -0.84355, -0.048509, -0.301789, 4.170682) + + val xMean = Array(5.843, 3.057, 3.758, 1.199) + val xVariance = Array(0.6856, 0.1899, 3.116, 0.581) + + val testData = LogisticRegressionSuite.generateMultinomialLogisticInput( + weights, xMean, xVariance, true, nPoints, 42) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + val lr = new LogisticRegressionWithLBFGS().setIntercept(true).setNumClasses(3) + lr.optimizer.setConvergenceTol(1E-15).setNumIterations(200) + + val model = lr.run(testRDD) + + /** + * The following is the instruction to reproduce the model using R's glmnet package. + * + * First of all, using the following scala code to save the data into `path`. + * + * testRDD.map(x => x.label+ ", " + x.features(0) + ", " + x.features(1) + ", " + + * x.features(2) + ", " + x.features(3)).saveAsTextFile("path") + * + * Using the following R code to load the data and train the model using glmnet package. + * + * library("glmnet") + * data <- read.csv("path", header=FALSE) + * label = factor(data$V1) + * features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) + * weights = coef(glmnet(features,label, family="multinomial", alpha = 0, lambda = 0)) + * + * The model weights of mutinomial logstic regression in R have `K` set of linear predictors + * for `K` classes classification problem; however, only `K-1` set is required if the first + * outcome is chosen as a "pivot", and the other `K-1` outcomes are separately regressed against + * the pivot outcome. This can be done by subtracting the first weights from those `K-1` set + * weights. The mathematical discussion and proof can be found here: + * http://en.wikipedia.org/wiki/Multinomial_logistic_regression + * + * weights1 = weights$`1` - weights$`0` + * weights2 = weights$`2` - weights$`0` + * + * > weights1 + * 5 x 1 sparse Matrix of class "dgCMatrix" + * s0 + * 2.6228269 + * data.V2 -0.5837166 + * data.V3 0.9285260 + * data.V4 -0.3783612 + * data.V5 -0.8123411 + * > weights2 + * 5 x 1 sparse Matrix of class "dgCMatrix" + * s0 + * 4.11197445 + * data.V2 -0.16918650 + * data.V3 -0.81104784 + * data.V4 -0.06463799 + * data.V5 -0.29198337 + */ + + val weightsR = Vectors.dense(Array( + -0.5837166, 0.9285260, -0.3783612, -0.8123411, 2.6228269, + -0.1691865, -0.811048, -0.0646380, -0.2919834, 4.1119745)) + + assert(model.weights ~== weightsR relTol 0.05) + + val validationData = LogisticRegressionSuite.generateMultinomialLogisticInput( + weights, xMean, xVariance, true, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + // The validation accuracy is not good since this model (even the original weights) doesn't have + // very steep curve in logistic function so that when we draw samples from distribution, it's + // very easy to assign to another labels. However, this prediction result is consistent to R. + validatePrediction(model.predict(validationRDD.map(_.features)).collect(), validationData, 0.47) + + } + } class LogisticRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { From dca6faa29a8dd805cf364ed2683efaf7928f2112 Mon Sep 17 00:00:00 2001 From: seayi <405078363@qq.com> Date: Mon, 2 Feb 2015 16:06:52 -0800 Subject: [PATCH 017/578] [SPARK-5195][sql]Update HiveMetastoreCatalog.scala(override the MetastoreRelation's sameresult method only compare databasename and table name) override the MetastoreRelation's sameresult method only compare databasename and table name because in previous : cache table t1; select count(*) from t1; it will read data from memory but the sql below will not,instead it read from hdfs: select count(*) from t1 t; because cache data is keyed by logical plan and compare with sameResult ,so when table with alias the same table 's logicalplan is not the same logical plan with out alias so modify the sameresult method only compare databasename and table name Author: seayi <405078363@qq.com> Author: Michael Armbrust Closes #3898 from seayi/branch-1.2 and squashes the following commits: 8f0c7d2 [seayi] Update CachedTableSuite.scala a277120 [seayi] Update HiveMetastoreCatalog.scala 8d910aa [seayi] Update HiveMetastoreCatalog.scala --- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 9 +++++++++ .../org/apache/spark/sql/hive/CachedTableSuite.scala | 6 ++++++ 2 files changed, 15 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 1a49f09bd9988..d910ee950904d 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 @@ -519,6 +519,15 @@ private[hive] case class MetastoreRelation } ) + /** Only compare database and tablename, not alias. */ + override def sameResult(plan: LogicalPlan): Boolean = { + plan match { + case mr: MetastoreRelation => + mr.databaseName == databaseName && mr.tableName == tableName + case _ => false + } + } + val tableDesc = HiveShim.getTableDesc( Class.forName( hiveQlTable.getSerializationLib, 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 61e5117feab10..7c8b5205e239e 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 @@ -64,6 +64,12 @@ class CachedTableSuite extends QueryTest { sql("SELECT * FROM src"), preCacheResults) + assertCached(sql("SELECT * FROM src s")) + + checkAnswer( + sql("SELECT * FROM src s"), + preCacheResults) + uncacheTable("src") assertCached(sql("SELECT * FROM src"), 0) } From 8aa3cfff661753d6d87a8d9a87373d403436dd92 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 2 Feb 2015 16:55:36 -0800 Subject: [PATCH 018/578] [SPARK-5514] DataFrame.collect should call executeCollect Author: Reynold Xin Closes #4313 from rxin/SPARK-5514 and squashes the following commits: e34e91b [Reynold Xin] [SPARK-5514] DataFrame.collect should call executeCollect --- 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 1096e396591df..5d42d4428d09e 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 @@ -541,7 +541,7 @@ class DataFrame protected[sql]( /** * Returns an array that contains all of [[Row]]s in this [[DataFrame]]. */ - override def collect(): Array[Row] = rdd.collect() + override def collect(): Array[Row] = queryExecution.executedPlan.executeCollect() /** * Returns a Java list that contains all of [[Row]]s in this [[DataFrame]]. From f133dece569f7faedb06a0da6e9f86c5d615e9be Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 2 Feb 2015 17:02:29 -0800 Subject: [PATCH 019/578] [SPARK-5534] [graphx] Graph getStorageLevel fix This fixes getStorageLevel for EdgeRDDImpl and VertexRDDImpl (and therefore for Graph). See code example on JIRA which failed before but works with this patch: [https://issues.apache.org/jira/browse/SPARK-5534] (The added unit tests also failed before but work with this fix.) Note: I used partitionsRDD, assuming that getStorageLevel will only be called on the driver. CC: mengxr (related to LDA PR), rxin ankurdave Thanks in advance! Author: Joseph K. Bradley Closes #4317 from jkbradley/graphx-storagelevel and squashes the following commits: 1c21e49 [Joseph K. Bradley] made graph getStorageLevel test more robust 18d64ca [Joseph K. Bradley] Added tests for getStorageLevel in VertexRDDSuite, EdgeRDDSuite, GraphSuite 17b488b [Joseph K. Bradley] overrode getStorageLevel in Vertex/EdgeRDDImpl to use partitionsRDD --- .../spark/graphx/impl/EdgeRDDImpl.scala | 2 + .../spark/graphx/impl/VertexRDDImpl.scala | 2 + .../apache/spark/graphx/EdgeRDDSuite.scala | 37 +++++++++++++++++++ .../org/apache/spark/graphx/GraphSuite.scala | 15 ++++++++ .../apache/spark/graphx/VertexRDDSuite.scala | 19 ++++++++-- 5 files changed, 71 insertions(+), 4 deletions(-) create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala 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 6c35d7029e078..56cb41661e300 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,6 +70,8 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] ( this } + override def getStorageLevel = partitionsRDD.getStorageLevel + override def checkpoint() = { partitionsRDD.checkpoint() } 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 3e4968d6c0d6f..6dad167fa7411 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,6 +71,8 @@ class VertexRDDImpl[VD] private[graphx] ( this } + override def getStorageLevel = partitionsRDD.getStorageLevel + override def checkpoint() = { partitionsRDD.checkpoint() } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala new file mode 100644 index 0000000000000..eb1dbe52c2fda --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx + +import org.scalatest.FunSuite + +import org.apache.spark.storage.StorageLevel + +class EdgeRDDSuite extends FunSuite with LocalSparkContext { + + test("cache, getStorageLevel") { + // test to see if getStorageLevel returns correct value after caching + withSpark { sc => + val verts = sc.parallelize(List((0L, 0), (1L, 1), (1L, 2), (2L, 3), (2L, 3), (2L, 3))) + val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) + assert(edges.getStorageLevel == StorageLevel.NONE) + edges.cache() + assert(edges.getStorageLevel == StorageLevel.MEMORY_ONLY) + } + } + +} 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 59a57ba7a33f1..b61d9f0fbe5e4 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -25,6 +25,7 @@ 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 class GraphSuite extends FunSuite with LocalSparkContext { @@ -390,6 +391,20 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } + test("cache, getStorageLevel") { + // test to see if getStorageLevel returns correct value + withSpark { sc => + val verts = sc.parallelize(List((1: VertexId, "a"), (2: VertexId, "b")), 1) + val edges = sc.parallelize(List(Edge(1, 2, 0), Edge(2, 1, 0)), 2) + val graph = Graph(verts, edges, "", StorageLevel.MEMORY_ONLY, StorageLevel.MEMORY_ONLY) + // Note: Before caching, graph.vertices is cached, but graph.edges is not (but graph.edges' + // parent RDD is cached). + graph.cache() + assert(graph.vertices.getStorageLevel == StorageLevel.MEMORY_ONLY) + assert(graph.edges.getStorageLevel == StorageLevel.MEMORY_ONLY) + } + } + test("non-default number of edge partitions") { val n = 10 val defaultParallelism = 3 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 42d3f21dbae98..131959cea3ef7 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -17,12 +17,11 @@ package org.apache.spark.graphx -import org.apache.spark.SparkContext -import org.apache.spark.graphx.Graph._ -import org.apache.spark.graphx.impl.EdgePartition -import org.apache.spark.rdd._ import org.scalatest.FunSuite +import org.apache.spark.SparkContext +import org.apache.spark.storage.StorageLevel + class VertexRDDSuite extends FunSuite with LocalSparkContext { def vertices(sc: SparkContext, n: Int) = { @@ -110,4 +109,16 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { } } + test("cache, getStorageLevel") { + // test to see if getStorageLevel returns correct value after caching + withSpark { sc => + val verts = sc.parallelize(List((0L, 0), (1L, 1), (1L, 2), (2L, 3), (2L, 3), (2L, 3))) + val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) + val rdd = VertexRDD(verts, edges, 0, (a: Int, b: Int) => a + b) + assert(rdd.getStorageLevel == StorageLevel.NONE) + rdd.cache() + assert(rdd.getStorageLevel == StorageLevel.MEMORY_ONLY) + } + } + } From ef65cf09b04f915ab463a6d3bac12795318897f2 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 2 Feb 2015 17:10:01 -0800 Subject: [PATCH 020/578] [SPARK-5540] hide ALS.solveLeastSquares This method survived the code review and it has been there since v1.1.0. It exposes jblas types. Let's remove it from the public API. I think no one calls it directly. Author: Xiangrui Meng Closes #4318 from mengxr/SPARK-5540 and squashes the following commits: 586ade6 [Xiangrui Meng] hide ALS.solveLeastSquares --- .../scala/org/apache/spark/mllib/recommendation/ALS.scala | 2 +- project/MimaExcludes.scala | 4 ++++ 2 files changed, 5 insertions(+), 1 deletion(-) 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 5f84677be238d..a5ffe888ca880 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 @@ -615,7 +615,7 @@ class ALS private ( * Given A^T A and A^T b, find the x minimising ||Ax - b||_2, possibly subject * to nonnegativity constraints if `nonnegative` is true. */ - def solveLeastSquares(ata: DoubleMatrix, atb: DoubleMatrix, + private def solveLeastSquares(ata: DoubleMatrix, atb: DoubleMatrix, ws: NNLS.Workspace): Array[Double] = { if (!nonnegative) { Solve.solvePositive(ata, atb).data diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 45be1db9a5ebe..78de1f0652741 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -66,6 +66,10 @@ object MimaExcludes { "org.apache.spark.mllib.linalg.Matrix.isTransposed"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.linalg.Matrix.foreachActive") + ) ++ Seq( + // SPARK-5540 + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.recommendation.ALS.solveLeastSquares") ) ++ Seq( // SPARK-3325 ProblemFilters.exclude[MissingMethodProblem]( From cfea30037ff4ac7e386a1478e7dce07ca3bb9072 Mon Sep 17 00:00:00 2001 From: Jacek Lewandowski Date: Mon, 2 Feb 2015 17:18:54 -0800 Subject: [PATCH 021/578] Spark 3883: SSL support for HttpServer and Akka SPARK-3883: SSL support for Akka connections and Jetty based file servers. This story introduced the following changes: - Introduced SSLOptions object which holds the SSL configuration and can build the appropriate configuration for Akka or Jetty. SSLOptions can be created by parsing SparkConf entries at a specified namespace. - SSLOptions is created and kept by SecurityManager - All Akka actor address creation snippets based on interpolated strings were replaced by a dedicated methods from AkkaUtils. Those methods select the proper Akka protocol - whether akka.tcp or akka.ssl.tcp - Added tests cases for AkkaUtils, FileServer, SSLOptions and SecurityManager - Added a way to use node local SSL configuration by executors and driver in standalone mode. It can be done by specifying spark.ssl.useNodeLocalConf in SparkConf. - Made CoarseGrainedExecutorBackend not overwrite the settings which are executor startup configuration - they are passed anyway from Worker Refer to https://github.com/apache/spark/pull/3571 for discussion and details Author: Jacek Lewandowski Author: Jacek Lewandowski Closes #3571 from jacek-lewandowski/SPARK-3883-master and squashes the following commits: 9ef4ed1 [Jacek Lewandowski] Merge pull request #2 from jacek-lewandowski/SPARK-3883-docs2 fb31b49 [Jacek Lewandowski] SPARK-3883: Added SSL setup documentation 2532668 [Jacek Lewandowski] SPARK-3883: Refactored AkkaUtils.protocol method to not use Try 90a8762 [Jacek Lewandowski] SPARK-3883: Refactored methods to resolve Akka address and made it possible to easily configure multiple communication layers for SSL 72b2541 [Jacek Lewandowski] SPARK-3883: A reference to the fallback SSLOptions can be provided when constructing SSLOptions 93050f4 [Jacek Lewandowski] SPARK-3883: SSL support for HttpServer and Akka --- .../scala/org/apache/spark/HttpServer.scala | 11 +- .../scala/org/apache/spark/SSLOptions.scala | 178 ++++++++++++++++ .../org/apache/spark/SecurityManager.scala | 100 ++++++++- .../scala/org/apache/spark/SparkConf.scala | 1 + .../spark/broadcast/HttpBroadcast.scala | 1 + .../spark/deploy/ApplicationDescription.scala | 9 + .../org/apache/spark/deploy/Client.scala | 5 +- .../spark/deploy/DriverDescription.scala | 8 + .../spark/deploy/client/AppClient.scala | 7 +- .../apache/spark/deploy/master/Master.scala | 8 +- .../spark/deploy/worker/ExecutorRunner.scala | 2 +- .../apache/spark/deploy/worker/Worker.scala | 67 +++++- .../CoarseGrainedExecutorBackend.scala | 16 +- .../cluster/SimrSchedulerBackend.scala | 4 +- .../cluster/SparkDeploySchedulerBackend.scala | 5 +- .../mesos/CoarseMesosSchedulerBackend.scala | 5 +- .../org/apache/spark/util/AkkaUtils.scala | 36 +++- .../scala/org/apache/spark/util/Utils.scala | 20 +- core/src/test/resources/keystore | Bin 0 -> 2247 bytes core/src/test/resources/truststore | Bin 0 -> 957 bytes core/src/test/resources/untrusted-keystore | Bin 0 -> 2246 bytes .../org/apache/spark/FileServerSuite.scala | 90 ++++++++ .../apache/spark/MapOutputTrackerSuite.scala | 2 +- .../org/apache/spark/SSLOptionsSuite.scala | 123 +++++++++++ .../org/apache/spark/SSLSampleConfigs.scala | 55 +++++ .../apache/spark/SecurityManagerSuite.scala | 50 ++++- .../org/apache/spark/deploy/ClientSuite.scala | 1 + .../spark/deploy/master/MasterSuite.scala | 26 ++- .../spark/deploy/worker/WorkerSuite.scala | 57 +++++ .../apache/spark/util/AkkaUtilsSuite.scala | 197 ++++++++++++++++-- docs/configuration.md | 80 +++++++ docs/security.md | 24 +++ .../spark/repl/ExecutorClassLoader.scala | 11 +- .../receiver/ReceiverSupervisorImpl.scala | 8 +- .../spark/deploy/yarn/ApplicationMaster.scala | 4 +- .../spark/deploy/yarn/YarnAllocator.scala | 7 +- 36 files changed, 1145 insertions(+), 73 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/SSLOptions.scala create mode 100644 core/src/test/resources/keystore create mode 100644 core/src/test/resources/truststore create mode 100644 core/src/test/resources/untrusted-keystore create mode 100644 core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala create mode 100644 core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index fa22787ce7ea3..09a9ccc226721 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -19,6 +19,7 @@ package org.apache.spark import java.io.File +import org.eclipse.jetty.server.ssl.SslSocketConnector import org.eclipse.jetty.util.security.{Constraint, Password} import org.eclipse.jetty.security.authentication.DigestAuthenticator import org.eclipse.jetty.security.{ConstraintMapping, ConstraintSecurityHandler, HashLoginService} @@ -72,7 +73,10 @@ private[spark] class HttpServer( */ private def doStart(startPort: Int): (Server, Int) = { val server = new Server() - val connector = new SocketConnector + + val connector = securityManager.fileServerSSLOptions.createJettySslContextFactory() + .map(new SslSocketConnector(_)).getOrElse(new SocketConnector) + connector.setMaxIdleTime(60 * 1000) connector.setSoLingerTime(-1) connector.setPort(startPort) @@ -149,13 +153,14 @@ private[spark] class HttpServer( } /** - * Get the URI of this HTTP server (http://host:port) + * Get the URI of this HTTP server (http://host:port or https://host:port) */ def uri: String = { if (server == null) { throw new ServerStateException("Server is not started") } else { - "http://" + Utils.localIpAddress + ":" + port + val scheme = if (securityManager.fileServerSSLOptions.enabled) "https" else "http" + s"$scheme://${Utils.localIpAddress}:$port" } } } diff --git a/core/src/main/scala/org/apache/spark/SSLOptions.scala b/core/src/main/scala/org/apache/spark/SSLOptions.scala new file mode 100644 index 0000000000000..2cdc167f85af0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/SSLOptions.scala @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import java.io.File + +import com.typesafe.config.{Config, ConfigFactory, ConfigValueFactory} +import org.eclipse.jetty.util.ssl.SslContextFactory + +/** + * SSLOptions class is a common container for SSL configuration options. It offers methods to + * generate specific objects to configure SSL for different communication protocols. + * + * SSLOptions is intended to provide the maximum common set of SSL settings, which are supported + * by the protocol, which it can generate the configuration for. Since Akka doesn't support client + * authentication with SSL, SSLOptions cannot support it either. + * + * @param enabled enables or disables SSL; if it is set to false, the rest of the + * settings are disregarded + * @param keyStore a path to the key-store file + * @param keyStorePassword a password to access the key-store file + * @param keyPassword a password to access the private key in the key-store + * @param trustStore a path to the trust-store file + * @param trustStorePassword a password to access the trust-store file + * @param protocol SSL protocol (remember that SSLv3 was compromised) supported by Java + * @param enabledAlgorithms a set of encryption algorithms to use + */ +private[spark] case class SSLOptions( + enabled: Boolean = false, + keyStore: Option[File] = None, + keyStorePassword: Option[String] = None, + keyPassword: Option[String] = None, + trustStore: Option[File] = None, + trustStorePassword: Option[String] = None, + protocol: Option[String] = None, + enabledAlgorithms: Set[String] = Set.empty) { + + /** + * Creates a Jetty SSL context factory according to the SSL settings represented by this object. + */ + def createJettySslContextFactory(): Option[SslContextFactory] = { + if (enabled) { + val sslContextFactory = new SslContextFactory() + + keyStore.foreach(file => sslContextFactory.setKeyStorePath(file.getAbsolutePath)) + trustStore.foreach(file => sslContextFactory.setTrustStore(file.getAbsolutePath)) + keyStorePassword.foreach(sslContextFactory.setKeyStorePassword) + trustStorePassword.foreach(sslContextFactory.setTrustStorePassword) + keyPassword.foreach(sslContextFactory.setKeyManagerPassword) + protocol.foreach(sslContextFactory.setProtocol) + sslContextFactory.setIncludeCipherSuites(enabledAlgorithms.toSeq: _*) + + Some(sslContextFactory) + } else { + None + } + } + + /** + * Creates an Akka configuration object which contains all the SSL settings represented by this + * object. It can be used then to compose the ultimate Akka configuration. + */ + def createAkkaConfig: Option[Config] = { + import scala.collection.JavaConversions._ + if (enabled) { + Some(ConfigFactory.empty() + .withValue("akka.remote.netty.tcp.security.key-store", + ConfigValueFactory.fromAnyRef(keyStore.map(_.getAbsolutePath).getOrElse(""))) + .withValue("akka.remote.netty.tcp.security.key-store-password", + ConfigValueFactory.fromAnyRef(keyStorePassword.getOrElse(""))) + .withValue("akka.remote.netty.tcp.security.trust-store", + ConfigValueFactory.fromAnyRef(trustStore.map(_.getAbsolutePath).getOrElse(""))) + .withValue("akka.remote.netty.tcp.security.trust-store-password", + ConfigValueFactory.fromAnyRef(trustStorePassword.getOrElse(""))) + .withValue("akka.remote.netty.tcp.security.key-password", + ConfigValueFactory.fromAnyRef(keyPassword.getOrElse(""))) + .withValue("akka.remote.netty.tcp.security.random-number-generator", + ConfigValueFactory.fromAnyRef("")) + .withValue("akka.remote.netty.tcp.security.protocol", + ConfigValueFactory.fromAnyRef(protocol.getOrElse(""))) + .withValue("akka.remote.netty.tcp.security.enabled-algorithms", + ConfigValueFactory.fromIterable(enabledAlgorithms.toSeq)) + .withValue("akka.remote.netty.tcp.enable-ssl", + ConfigValueFactory.fromAnyRef(true))) + } else { + None + } + } + + /** Returns a string representation of this SSLOptions with all the passwords masked. */ + override def toString: String = s"SSLOptions{enabled=$enabled, " + + s"keyStore=$keyStore, keyStorePassword=${keyStorePassword.map(_ => "xxx")}, " + + s"trustStore=$trustStore, trustStorePassword=${trustStorePassword.map(_ => "xxx")}, " + + s"protocol=$protocol, enabledAlgorithms=$enabledAlgorithms}" + +} + +private[spark] object SSLOptions extends Logging { + + /** Resolves SSLOptions settings from a given Spark configuration object at a given namespace. + * + * The following settings are allowed: + * $ - `[ns].enabled` - `true` or `false`, to enable or disable SSL respectively + * $ - `[ns].keyStore` - a path to the key-store file; can be relative to the current directory + * $ - `[ns].keyStorePassword` - a password to the key-store file + * $ - `[ns].keyPassword` - a password to the private key + * $ - `[ns].trustStore` - a path to the trust-store file; can be relative to the current + * directory + * $ - `[ns].trustStorePassword` - a password to the trust-store file + * $ - `[ns].protocol` - a protocol name supported by a particular Java version + * $ - `[ns].enabledAlgorithms` - a comma separated list of ciphers + * + * For a list of protocols and ciphers supported by particular Java versions, you may go to + * [[https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https Oracle + * blog page]]. + * + * You can optionally specify the default configuration. If you do, for each setting which is + * missing in SparkConf, the corresponding setting is used from the default configuration. + * + * @param conf Spark configuration object where the settings are collected from + * @param ns the namespace name + * @param defaults the default configuration + * @return [[org.apache.spark.SSLOptions]] object + */ + def parse(conf: SparkConf, ns: String, defaults: Option[SSLOptions] = None): SSLOptions = { + val enabled = conf.getBoolean(s"$ns.enabled", defaultValue = defaults.exists(_.enabled)) + + val keyStore = conf.getOption(s"$ns.keyStore").map(new File(_)) + .orElse(defaults.flatMap(_.keyStore)) + + val keyStorePassword = conf.getOption(s"$ns.keyStorePassword") + .orElse(defaults.flatMap(_.keyStorePassword)) + + val keyPassword = conf.getOption(s"$ns.keyPassword") + .orElse(defaults.flatMap(_.keyPassword)) + + val trustStore = conf.getOption(s"$ns.trustStore").map(new File(_)) + .orElse(defaults.flatMap(_.trustStore)) + + val trustStorePassword = conf.getOption(s"$ns.trustStorePassword") + .orElse(defaults.flatMap(_.trustStorePassword)) + + val protocol = conf.getOption(s"$ns.protocol") + .orElse(defaults.flatMap(_.protocol)) + + val enabledAlgorithms = conf.getOption(s"$ns.enabledAlgorithms") + .map(_.split(",").map(_.trim).filter(_.nonEmpty).toSet) + .orElse(defaults.map(_.enabledAlgorithms)) + .getOrElse(Set.empty) + + new SSLOptions( + enabled, + keyStore, + keyStorePassword, + keyPassword, + trustStore, + trustStorePassword, + protocol, + enabledAlgorithms) + } + +} + diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index ec82d09cd079b..88d35a4bacc6e 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -18,7 +18,11 @@ package org.apache.spark import java.net.{Authenticator, PasswordAuthentication} +import java.security.KeyStore +import java.security.cert.X509Certificate +import javax.net.ssl._ +import com.google.common.io.Files import org.apache.hadoop.io.Text import org.apache.spark.deploy.SparkHadoopUtil @@ -55,7 +59,7 @@ import org.apache.spark.network.sasl.SecretKeyHolder * Spark also has a set of admin acls (`spark.admin.acls`) which is a set of users/administrators * who always have permission to view or modify the Spark application. * - * Spark does not currently support encryption after authentication. + * Starting from version 1.3, Spark has partial support for encrypted connections with SSL. * * At this point spark has multiple communication protocols that need to be secured and * different underlying mechanisms are used depending on the protocol: @@ -67,8 +71,9 @@ import org.apache.spark.network.sasl.SecretKeyHolder * to connect to the server. There is no control of the underlying * authentication mechanism so its not clear if the password is passed in * plaintext or uses DIGEST-MD5 or some other mechanism. - * Akka also has an option to turn on SSL, this option is not currently supported - * but we could add a configuration option in the future. + * + * Akka also has an option to turn on SSL, this option is currently supported (see + * the details below). * * - HTTP for broadcast and file server (via HttpServer) -> Spark currently uses Jetty * for the HttpServer. Jetty supports multiple authentication mechanisms - @@ -77,8 +82,9 @@ import org.apache.spark.network.sasl.SecretKeyHolder * to authenticate using DIGEST-MD5 via a single user and the shared secret. * Since we are using DIGEST-MD5, the shared secret is not passed on the wire * in plaintext. - * We currently do not support SSL (https), but Jetty can be configured to use it - * so we could add a configuration option for this in the future. + * + * We currently support SSL (https) for this communication protocol (see the details + * below). * * The Spark HttpServer installs the HashLoginServer and configures it to DIGEST-MD5. * Any clients must specify the user and password. There is a default @@ -142,9 +148,40 @@ import org.apache.spark.network.sasl.SecretKeyHolder * authentication. Spark will then use that user to compare against the view acls to do * authorization. If not filter is in place the user is generally null and no authorization * can take place. + * + * Connection encryption (SSL) configuration is organized hierarchically. The user can configure + * the default SSL settings which will be used for all the supported communication protocols unless + * they are overwritten by protocol specific settings. This way the user can easily provide the + * common settings for all the protocols without disabling the ability to configure each one + * individually. + * + * All the SSL settings like `spark.ssl.xxx` where `xxx` is a particular configuration property, + * denote the global configuration for all the supported protocols. In order to override the global + * configuration for the particular protocol, the properties must be overwritten in the + * protocol-specific namespace. Use `spark.ssl.yyy.xxx` settings to overwrite the global + * configuration for particular protocol denoted by `yyy`. Currently `yyy` can be either `akka` for + * Akka based connections or `fs` for broadcast and file server. + * + * Refer to [[org.apache.spark.SSLOptions]] documentation for the list of + * options that can be specified. + * + * SecurityManager initializes SSLOptions objects for different protocols separately. SSLOptions + * object parses Spark configuration at a given namespace and builds the common representation + * of SSL settings. SSLOptions is then used to provide protocol-specific configuration like + * TypeSafe configuration for Akka or SSLContextFactory for Jetty. + * + * SSL must be configured on each node and configured for each component involved in + * communication using the particular protocol. In YARN clusters, the key-store can be prepared on + * the client side then distributed and used by the executors as the part of the application + * (YARN allows the user to deploy files before the application is started). + * In standalone deployment, the user needs to provide key-stores and configuration + * options for master and workers. In this mode, the user may allow the executors to use the SSL + * settings inherited from the worker which spawned that executor. It can be accomplished by + * setting `spark.ssl.useNodeLocalConf` to `true`. */ -private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging with SecretKeyHolder { +private[spark] class SecurityManager(sparkConf: SparkConf) + extends Logging with SecretKeyHolder { // key used to store the spark secret in the Hadoop UGI private val sparkSecretLookupKey = "sparkCookie" @@ -196,6 +233,57 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging with ) } + // the default SSL configuration - it will be used by all communication layers unless overwritten + private val defaultSSLOptions = SSLOptions.parse(sparkConf, "spark.ssl", defaults = None) + + // SSL configuration for different communication layers - they can override the default + // configuration at a specified namespace. The namespace *must* start with spark.ssl. + val fileServerSSLOptions = SSLOptions.parse(sparkConf, "spark.ssl.fs", Some(defaultSSLOptions)) + val akkaSSLOptions = SSLOptions.parse(sparkConf, "spark.ssl.akka", Some(defaultSSLOptions)) + + logDebug(s"SSLConfiguration for file server: $fileServerSSLOptions") + logDebug(s"SSLConfiguration for Akka: $akkaSSLOptions") + + val (sslSocketFactory, hostnameVerifier) = if (fileServerSSLOptions.enabled) { + val trustStoreManagers = + for (trustStore <- fileServerSSLOptions.trustStore) yield { + val input = Files.asByteSource(fileServerSSLOptions.trustStore.get).openStream() + + try { + val ks = KeyStore.getInstance(KeyStore.getDefaultType) + ks.load(input, fileServerSSLOptions.trustStorePassword.get.toCharArray) + + val tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm) + tmf.init(ks) + tmf.getTrustManagers + } finally { + input.close() + } + } + + lazy val credulousTrustStoreManagers = Array({ + logWarning("Using 'accept-all' trust manager for SSL connections.") + new X509TrustManager { + override def getAcceptedIssuers: Array[X509Certificate] = null + + override def checkClientTrusted(x509Certificates: Array[X509Certificate], s: String) {} + + override def checkServerTrusted(x509Certificates: Array[X509Certificate], s: String) {} + }: TrustManager + }) + + val sslContext = SSLContext.getInstance(fileServerSSLOptions.protocol.getOrElse("Default")) + sslContext.init(null, trustStoreManagers.getOrElse(credulousTrustStoreManagers), null) + + val hostVerifier = new HostnameVerifier { + override def verify(s: String, sslSession: SSLSession): Boolean = true + } + + (Some(sslContext.getSocketFactory), Some(hostVerifier)) + } else { + (None, None) + } + /** * Split a comma separated String, filter out any empty items, and return a Set of strings */ diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 4d4c69d42da98..13aa9960ac33a 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -370,6 +370,7 @@ private[spark] object SparkConf { isAkkaConf(name) || name.startsWith("spark.akka") || name.startsWith("spark.auth") || + name.startsWith("spark.ssl") || isSparkPortConf(name) } 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 ea98051532a0a..1444c0dd3d2d6 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -199,6 +199,7 @@ private[broadcast] object HttpBroadcast extends Logging { uc = new URL(url).openConnection() uc.setConnectTimeout(httpReadTimeout) } + Utils.setupSecureURLConnection(uc, securityManager) val in = { uc.setReadTimeout(httpReadTimeout) 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 65a1a8fd7e929..ae55b4ff40b74 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -28,5 +28,14 @@ private[spark] class ApplicationDescription( val user = System.getProperty("user.name", "") + def copy( + name: String = name, + maxCores: Option[Int] = maxCores, + memoryPerSlave: Int = memoryPerSlave, + command: Command = command, + appUiUrl: String = appUiUrl, + eventLogDir: Option[String] = eventLogDir): ApplicationDescription = + new ApplicationDescription(name, maxCores, memoryPerSlave, command, appUiUrl, eventLogDir) + override def toString: String = "ApplicationDescription(" + name + ")" } 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 7c1c831c248fc..38b3da0b13756 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -39,7 +39,8 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) val timeout = AkkaUtils.askTimeout(conf) override def preStart() = { - masterActor = context.actorSelection(Master.toAkkaUrl(driverArgs.master)) + masterActor = context.actorSelection( + Master.toAkkaUrl(driverArgs.master, AkkaUtils.protocol(context.system))) context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) @@ -161,7 +162,7 @@ object Client { "driverClient", Utils.localHostName(), 0, conf, new SecurityManager(conf)) // Verify driverArgs.master is a valid url so that we can use it in ClientActor safely - Master.toAkkaUrl(driverArgs.master) + Master.toAkkaUrl(driverArgs.master, AkkaUtils.protocol(actorSystem)) actorSystem.actorOf(Props(classOf[ClientActor], driverArgs, conf)) actorSystem.awaitTermination() 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 58c95dc4f9116..b056a19ce6598 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DriverDescription.scala @@ -25,5 +25,13 @@ private[spark] class DriverDescription( val command: Command) extends Serializable { + def copy( + jarUrl: String = jarUrl, + mem: Int = mem, + cores: Int = cores, + supervise: Boolean = supervise, + command: Command = command): DriverDescription = + new DriverDescription(jarUrl, mem, cores, supervise, command) + override def toString: String = s"DriverDescription (${command.mainClass})" } 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 39a7b0319b6a1..ffe940fbda2fb 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,7 +47,7 @@ private[spark] class AppClient( conf: SparkConf) extends Logging { - val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl) + val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl(_, AkkaUtils.protocol(actorSystem))) val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 @@ -107,8 +107,9 @@ private[spark] class AppClient( def changeMaster(url: String) { // activeMasterUrl is a valid Spark url since we receive it from master. activeMasterUrl = url - master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl)) - masterAddress = Master.toAkkaAddress(activeMasterUrl) + master = context.actorSelection( + Master.toAkkaUrl(activeMasterUrl, AkkaUtils.protocol(actorSystem))) + masterAddress = Master.toAkkaAddress(activeMasterUrl, AkkaUtils.protocol(actorSystem)) } private def isPossibleMaster(remoteUrl: Address) = { 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 d92d99310a583..5eeb9fe526248 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 @@ -860,9 +860,9 @@ private[spark] object Master extends Logging { * * @throws SparkException if the url is invalid */ - def toAkkaUrl(sparkUrl: String): String = { + def toAkkaUrl(sparkUrl: String, protocol: String): String = { val (host, port) = Utils.extractHostPortFromSparkUrl(sparkUrl) - "akka.tcp://%s@%s:%s/user/%s".format(systemName, host, port, actorName) + AkkaUtils.address(protocol, systemName, host, port, actorName) } /** @@ -870,9 +870,9 @@ private[spark] object Master extends Logging { * * @throws SparkException if the url is invalid */ - def toAkkaAddress(sparkUrl: String): Address = { + def toAkkaAddress(sparkUrl: String, protocol: String): Address = { val (host, port) = Utils.extractHostPortFromSparkUrl(sparkUrl) - Address("akka.tcp", systemName, host, port) + Address(protocol, systemName, host, port) } def startSystemAndActor( 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 acbdf0d8bd7bc..bc9f78b9e5c77 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 @@ -26,7 +26,7 @@ import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.apache.spark.{SparkConf, Logging} -import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} +import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged import org.apache.spark.util.logging.FileAppender 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 13599830123d0..b20f5c0c82895 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 @@ -31,8 +31,8 @@ import scala.util.Random import akka.actor._ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} -import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} +import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.deploy.{Command, ExecutorDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.deploy.worker.ui.WorkerWebUI @@ -93,7 +93,12 @@ private[spark] class Worker( var masterAddress: Address = null var activeMasterUrl: String = "" var activeMasterWebUiUrl : String = "" - val akkaUrl = "akka.tcp://%s@%s:%s/user/%s".format(actorSystemName, host, port, actorName) + val akkaUrl = AkkaUtils.address( + AkkaUtils.protocol(context.system), + actorSystemName, + host, + port, + actorName) @volatile var registered = false @volatile var connected = false val workerId = generateWorkerId() @@ -174,8 +179,9 @@ private[spark] class Worker( // activeMasterUrl it's a valid Spark url since we receive it from master. activeMasterUrl = url activeMasterWebUiUrl = uiUrl - master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl)) - masterAddress = Master.toAkkaAddress(activeMasterUrl) + master = context.actorSelection( + Master.toAkkaUrl(activeMasterUrl, AkkaUtils.protocol(context.system))) + masterAddress = Master.toAkkaAddress(activeMasterUrl, AkkaUtils.protocol(context.system)) connected = true // Cancel any outstanding re-registration attempts because we found a new master registrationRetryTimer.foreach(_.cancel()) @@ -347,10 +353,20 @@ private[spark] class Worker( }.toSeq } appDirectories(appId) = appLocalDirs - - val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_, - self, workerId, host, sparkHome, executorDir, akkaUrl, conf, appLocalDirs, - ExecutorState.LOADING) + val manager = new ExecutorRunner( + appId, + execId, + appDesc.copy(command = Worker.maybeUpdateSSLSettings(appDesc.command, conf)), + cores_, + memory_, + self, + workerId, + host, + sparkHome, + executorDir, + akkaUrl, + conf, + appLocalDirs, ExecutorState.LOADING) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ @@ -406,7 +422,14 @@ private[spark] class Worker( case LaunchDriver(driverId, driverDesc) => { logInfo(s"Asked to launch driver $driverId") - val driver = new DriverRunner(conf, driverId, workDir, sparkHome, driverDesc, self, akkaUrl) + val driver = new DriverRunner( + conf, + driverId, + workDir, + sparkHome, + driverDesc.copy(command = Worker.maybeUpdateSSLSettings(driverDesc.command, conf)), + self, + akkaUrl) drivers(driverId) = driver driver.start() @@ -523,10 +546,32 @@ private[spark] object Worker extends Logging { val securityMgr = new SecurityManager(conf) val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, conf = conf, securityManager = securityMgr) - val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl) + val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl(_, AkkaUtils.protocol(actorSystem))) actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory, masterAkkaUrls, systemName, actorName, workDir, conf, securityMgr), name = actorName) (actorSystem, boundPort) } + private[spark] def isUseLocalNodeSSLConfig(cmd: Command): Boolean = { + val pattern = """\-Dspark\.ssl\.useNodeLocalConf\=(.+)""".r + val result = cmd.javaOpts.collectFirst { + case pattern(_result) => _result.toBoolean + } + result.getOrElse(false) + } + + private[spark] def maybeUpdateSSLSettings(cmd: Command, conf: SparkConf): Command = { + val prefix = "spark.ssl." + val useNLC = "spark.ssl.useNodeLocalConf" + if (isUseLocalNodeSSLConfig(cmd)) { + val newJavaOpts = cmd.javaOpts + .filter(opt => !opt.startsWith(s"-D$prefix")) ++ + conf.getAll.collect { case (key, value) if key.startsWith(prefix) => s"-D$key=$value" } :+ + s"-D$useNLC=true" + cmd.copy(javaOpts = newJavaOpts) + } else { + cmd + } + } + } 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 823825302658c..bc72c8970319c 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -123,7 +123,11 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val executorConf = new SparkConf val port = executorConf.getInt("spark.executor.port", 0) val (fetcher, _) = AkkaUtils.createActorSystem( - "driverPropsFetcher", hostname, port, executorConf, new SecurityManager(executorConf)) + "driverPropsFetcher", + hostname, + port, + executorConf, + new SecurityManager(executorConf)) val driver = fetcher.actorSelection(driverUrl) val timeout = AkkaUtils.askTimeout(executorConf) val fut = Patterns.ask(driver, RetrieveSparkProps, timeout) @@ -132,7 +136,15 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { fetcher.shutdown() // Create SparkEnv using properties we fetched from the driver. - val driverConf = new SparkConf().setAll(props) + val driverConf = new SparkConf() + for ((key, value) <- props) { + // this is required for SSL in standalone mode + if (SparkConf.isExecutorStartupConf(key)) { + driverConf.setIfMissing(key, value) + } else { + driverConf.set(key, value) + } + } val env = SparkEnv.createExecutorEnv( driverConf, executorId, hostname, port, cores, isLocal = false) 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 ee10aa061f4e9..06786a59524e7 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 @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.{Path, FileSystem} 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, @@ -38,7 +39,8 @@ private[spark] class SimrSchedulerBackend( override def start() { super.start() - val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + val driverUrl = AkkaUtils.address( + AkkaUtils.protocol(actorSystem), SparkEnv.driverActorSystemName, sc.conf.get("spark.driver.host"), sc.conf.get("spark.driver.port"), 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 7eb87a564d6f5..d2e1680a5fd1b 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 @@ -21,7 +21,7 @@ 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.Utils +import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class SparkDeploySchedulerBackend( scheduler: TaskSchedulerImpl, @@ -46,7 +46,8 @@ private[spark] class SparkDeploySchedulerBackend( super.start() // The endpoint for executors to talk to us - val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + val driverUrl = AkkaUtils.address( + AkkaUtils.protocol(actorSystem), SparkEnv.driverActorSystemName, conf.get("spark.driver.host"), conf.get("spark.driver.port"), 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 5289661eb896b..0d1c2a916ca7f 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 @@ -31,7 +31,7 @@ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTas import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import org.apache.spark.util.Utils +import org.apache.spark.util.{Utils, AkkaUtils} /** * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds @@ -143,7 +143,8 @@ private[spark] class CoarseMesosSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + val driverUrl = AkkaUtils.address( + AkkaUtils.protocol(sc.env.actorSystem), SparkEnv.driverActorSystemName, conf.get("spark.driver.host"), conf.get("spark.driver.port"), 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 4c9b1e3c46f0f..3d9c6192ff7f7 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -20,6 +20,7 @@ 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 @@ -91,8 +92,11 @@ private[spark] object AkkaUtils extends Logging { val secureCookie = if (isAuthOn) secretKey else "" logDebug(s"In createActorSystem, requireCookie is: $requireCookie") - val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]).withFallback( - ConfigFactory.parseString( + val akkaSslConfig = securityManager.akkaSSLOptions.createAkkaConfig + .getOrElse(ConfigFactory.empty()) + + val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]) + .withFallback(akkaSslConfig).withFallback(ConfigFactory.parseString( s""" |akka.daemonic = on |akka.loggers = [""akka.event.slf4j.Slf4jLogger""] @@ -214,7 +218,7 @@ private[spark] object AkkaUtils extends Logging { val driverHost: String = conf.get("spark.driver.host", "localhost") val driverPort: Int = conf.getInt("spark.driver.port", 7077) Utils.checkHost(driverHost, "Expected hostname") - val url = s"akka.tcp://$driverActorSystemName@$driverHost:$driverPort/user/$name" + val url = address(protocol(actorSystem), driverActorSystemName, driverHost, driverPort, name) val timeout = AkkaUtils.lookupTimeout(conf) logInfo(s"Connecting to $name: $url") Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) @@ -228,9 +232,33 @@ private[spark] object AkkaUtils extends Logging { actorSystem: ActorSystem): ActorRef = { val executorActorSystemName = SparkEnv.executorActorSystemName Utils.checkHost(host, "Expected hostname") - val url = s"akka.tcp://$executorActorSystemName@$host:$port/user/$name" + val url = address(protocol(actorSystem), executorActorSystemName, host, port, name) val timeout = AkkaUtils.lookupTimeout(conf) logInfo(s"Connecting to $name: $url") Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) } + + def protocol(actorSystem: ActorSystem): String = { + val akkaConf = actorSystem.settings.config + val sslProp = "akka.remote.netty.tcp.enable-ssl" + protocol(akkaConf.hasPath(sslProp) && akkaConf.getBoolean(sslProp)) + } + + def protocol(ssl: Boolean = false): String = { + if (ssl) { + "akka.ssl.tcp" + } else { + "akka.tcp" + } + } + + def address( + protocol: String, + systemName: String, + host: String, + port: Any, + actorName: String): String = { + s"$protocol://$systemName@$host:$port/user/$actorName" + } + } 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 31850b50bdba7..e9f2aed9ffbea 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -21,8 +21,9 @@ import java.io._ import java.lang.management.ManagementFactory import java.net._ import java.nio.ByteBuffer -import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadFactory, ThreadPoolExecutor} -import java.util.{Locale, Properties, Random, UUID} +import java.util.{Properties, Locale, Random, UUID} +import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} +import javax.net.ssl.HttpsURLConnection import scala.collection.JavaConversions._ import scala.collection.Map @@ -575,6 +576,7 @@ private[spark] object Utils extends Logging { logDebug("fetchFile not using security") uc = new URL(url).openConnection() } + Utils.setupSecureURLConnection(uc, securityMgr) val timeout = conf.getInt("spark.files.fetchTimeout", 60) * 1000 uc.setConnectTimeout(timeout) @@ -1820,6 +1822,20 @@ private[spark] object Utils extends Logging { PropertyConfigurator.configure(pro) } + /** + * If the given URL connection is HttpsURLConnection, it sets the SSL socket factory and + * the host verifier from the given security manager. + */ + def setupSecureURLConnection(urlConnection: URLConnection, sm: SecurityManager): URLConnection = { + urlConnection match { + case https: HttpsURLConnection => + sm.sslSocketFactory.foreach(https.setSSLSocketFactory) + sm.hostnameVerifier.foreach(https.setHostnameVerifier) + https + case connection => connection + } + } + def invoke( clazz: Class[_], obj: AnyRef, diff --git a/core/src/test/resources/keystore b/core/src/test/resources/keystore new file mode 100644 index 0000000000000000000000000000000000000000..f8310e39ba1e07c7559e5724f522c06ca47f5787 GIT binary patch literal 2247 zcmchYc{J1uAI9f5Gt8i|P1&wBlPn{@Y0#AI+9uMFEKS|9bCtJ0{)TM zX}X`!>67%Z@GWAuh_Ft8Kt!M*6TS&bii+!r0&qYVDgyu_5I7UgZ~dLDFFKbbe4NJ> z8h*OOS9N;OWepcs>OGoE=zuk2V{*wI)Z_e}>Z@^h>_v>&7v=eK%-(UR|F2)6ifSZm zB{$HW8@=O<)XbNH;_wk=FeM$Bjfx=?s>^D9Zr#6WTzln-!DYFW$=EZVD(3K>jS96w z-f`E>K|`eE2uJ#EUlg%5TBw+iWJz5iySD{o5{Ul(C)@&^f0t&B->D-6`JH0%@@;s@ zjMngAG004|Ao+Z!S-#o>U%j;FhYZ-6+Sr%89Pdns-=>tDGRpP7IeZ?->$L-DANQrdWKTa;0 zluk^040v;S@5DL7!>L;M8&vwCZAjLK*vpGqYx~-VYZs+3mb>>RYt{axVJiP>$3Y|} zz*eB_iPWr6Tc->>lU_2dn3A0UZrr!p`Ly9<%d%n&X#yvmACj3LY^dq?e=FaqJ!`fkB(%M!T4bA&bADq%;%h1-L)a{YF#I>*n5; zXt7P-gtHyc-1yawv9;te^CLkar+P2}c`q>)=Cwt62se##oHtS0qXNZJW*YMjnTYe| z1W+lZjRJNxO-J~$3RTEpZS0TIp!w}_nOBXJ>YGGE1!=AA8t_d9vGYswy&Plo@H7BgF?Rr;IN}~I_OZ5AE?Xc>e zduB^$Zl^0#&4F^xR^%KivoAj*R)05F<5T{9#!cLae!D`i$@(i5VdKPs;vD6ow-Pa0 zmfi>Du&L;*jI`*2mxHbvy2C)a@EF^M?KmG1vd1J;hO<00q{Ea0+F@!vpWZUoIem_pk7QGFApU$XWEZyP!S$ zy2#@wW_snjaE*mkb%s`~Ur~W0n&f9pXRDv9^a`mg8=d@ECh_CSHJ#du`ah#SxB9W1 z-pb!}YxjkPKt*mOLDS3IRD~MerwrgiggePgez$MStIAG30Vaz+z znP5Kohc09y*(6S=bZ0;6*@CtY{i164xeX%S_5pQfMC@N!nO0wH%Xtuc`Uu_?pnpJ3cKSbR^Nesz@fkeg8K_ zTJ35i8GW{Bgm&Uy_A)8Ar>TC80v~KUm51AD4|t?pz-*InPU(xFwfe7aK@- z&`sYxAFsV#57|9oUhntf^%Ms6?u7Ik>%Ezi3GO{ccd$K+?Z=*8-c9HP#s zOrXfNhRKwkI)1xzMcTvsXTrA@mGdnR9#$e8BDNj1us(Rjp^I@0JHr=J9h*keYb@4` zEbRm4_$*5QRieRoP)JDy&5{>VVsBkDRg>0s_n`ei%|c)v!>9n^H~H}eLNotr3B^*= z{jjq6yqK``^Vn@D41fS-8lW1evegh3Nw^$*J9JmC0%UznnvdunEUWXJm7jB#`FDt0 zo!SZ!fDlyn__i1(A!8e5JfGHGYq#O59PYAPUd^G9FA4GLhdQU@y3_KivZwp9e#sjZ z1)6E~Py4647bJ-*g&tbSSqXlAo`|VY0-a#1EcqqpmY_MpMl0!c@gNTqd?DPoAC6{F zX#Czw>93?(`H$;NoCL46jICSrt~xR}wNV^~dn<|;)#U(7hP$oGOzs-~AvLGK&g0zM zw^?bP=4NGFJ3;N}QsShyk0XSs?LW@9d~Sxj?AVZ{pOtdaJ7!gP>dq$(AFE$ic(QF_ zMtzQi_+9hZzoOnZ-qFt&6a;ssvcmgs))|WiZVphZ+9n#oo9dzuTg|>3rcC)~XD4Ad ZwPrecI^>>DS9jrYAM6Ze@sCox{su3o&c*-$ literal 0 HcmV?d00001 diff --git a/core/src/test/resources/truststore b/core/src/test/resources/truststore new file mode 100644 index 0000000000000000000000000000000000000000..a6b1d46e1f391995553771665934518114892962 GIT binary patch literal 957 zcmezO_TO6u1_mY|W(3omIr+(nIT`uIB|s66PaJ187+53pObsj<7?`UKnwZNCnwa7j zFf%bSF|h<%`7Sl!W#iOp^Jx3d%gD&h%3zRVC}6bQhKm@;iSrto7#bK@7#SNH7@I@^xuyn&24+w$on6z!sDvCGjI0dIO^o~u z22G4yOihf849&hp=Zrag|9n~Gc#H3$vhv=@pKm=9>UvU(ZZwCjnIky!nB{NDyiMQB zBZO<_+{laCF!_t`)YJQR{_Ib&x!G3fds+PIs-~1XLS?NdeqFL|Ki_tDnb_pb)#>@( zTD|J;>oi2PIZoZR+nId!%3~GY`%wb7osA|uXbd}Tx=!cX%NWDQox39rO%?vMzHf?~ z=1swLR-RoC+D>nNbu4gY&$gL|-rloc_5R@2ZL8S5%Xot9o!zH-pDYd6dHL#Xw%@*+ zy&LU+nmT%>zI)Xlb76*?{++DCc`NfA&lk?~bl&)QW6#lb?w*%6eoUWUYC4NO`N6ZZ z?8-A|%!_eU4)c+*d(ZiH(T7{hnV1hF?DgUa_CL zx_EQkYw>r1g2;ggOjE!>WMr_A)k`{P(!x=_M=);x=K055cF6ji^!W09N=!ueTVn;U z6EA8{^-Nzg`Ng@3($jCUF&@ae8^o zyyq)UdNa4Ognk!%pnCm3|2v5#(^qdW|F_DnC)CMo)7|XjC#6>R9=pT-w&U3U_Jbb| zump?#5z4RO57rKM+MK!exniP|!?*K7M~h9*`B~JdzSntD@o>*t-I;G@<(}y1FS)RE zvl(0Zzf0L`j^5p4uvCF<+hNhw literal 0 HcmV?d00001 diff --git a/core/src/test/resources/untrusted-keystore b/core/src/test/resources/untrusted-keystore new file mode 100644 index 0000000000000000000000000000000000000000..6015b02caa12817721fca3a83c8e58338b6d9aeb GIT binary patch literal 2246 zcmchYS5VUl7sc~W0)Ye-=~4xxg#Zav38z3~KrxBxU2`W=0g8?MR?KmjE9H~?^gpj7C>T*;?*E%{fv1t#2A zwmzrvHqay1&t+Fi_Z+)w1n4H+Y%AMXJ&rSoCSXIQW|Hr+fhOC{~-z zRlY#ibR-jPTxR#)AT!BEO?ezTUwe>&ocp&K#iX4G4x}&l;Dj))1SC+yocOHPKcyd#4h$$Ysx{PW2wSrMAGjmd{> zmZKc2Dyf?4ujwLhFCx*5Hx~U|Z)4cF@ukC(ds5*mRqTjWZQ!rEzURPjf;fj^#543K+ z!ssm8re)pqzo9BLWxDF++xd2(u`*)aHlZZpG29)-s8jv^L!<=2n_lp*^DBVw#MvPDYjjU59hk-*=5ySF?;vN-zNZGbmhBK9h zT^c^=AHxzXuw5SuT_j1a9=(n&J?~VzvG}qlOYj8&6PtlOG82v{Ls5creeldHDY=n} z6NS98Hi+3%9-T{Nl<}Y`*yL)P*pyGtkkOKUx$aHwg>deSRCRDu3S_Bt*qYqP;^=vW z>Sial$32p`22-|}%Wtf09n6kxR~g&{ z;kf$;?1pyH#legE8IIWJMw1!v9A-Bq$dIyn#d=DTZcOxsMZN<8A29U{XY8FE^^?*t(;Wc9KRLxuFXiP-G z>e>s*`P%O)W#bh}MOrv+{^3tN%a_tNb9D8h>-eF{WYv1B{AL3=1%=#OJh|?KR$7@~ zM1%siVBJLAn~Pz$VnIy*@}LG*a&*qN<%mI`ZQi8>jN6Bzl5(ULw7YAmA5UoEllgUDu` zAewV|((5eVYOO+`<3sL2d)BS_Mok%KAB6q1*Jc7zF$H5nC_yMo`6_U&c?)aPk4b$z zk=~H8I#D$8!fS_4;uOVeZ*ca0M>wc+>X_;202 zht3{zS6QTZ`z}%LI2uqFW?YIRHI*33>s?|kWxkxSkI|mB{+aCe>R$D#3C_eDcm?~4 z04*s~%RJne@gR@YM1zFG)0DX;`R-ZeVAqWJizOR@=|SzH@w)|jt`!(`wOVk zU(Gc+3x6OSUy2;rv#hrK`;koo{3~F+?Lz<_V{4;1#8arvZ&0V8@AqY z%gOfdX610i{YAs|(!9Q+SL{EgzRT#t9JFQIGR2S`6VaKXbv2cJ2H)BOR*n|BK8B>J zu4T{_d4E~Njm&*(N{Lz*K3`qRFrPLYtX(hoRRuEpLT4Gjc z7p!#ttliZeFa!Vrd9r93wA5ijq>e&GphCV$=V%&Tr1x)APN~~@unO~heb4>d#KTM- znh1b2&I@^pv~P&tiq}$Qc2}5{Y1K+f=lfqETn=T3E>iK(`_W8&jcZK}2tO)n$}*DS zW9s~&EeO=mYgrtHq(vr!#h3NAy`7YJ8V0q&`(N)hb2~hoF87PZRGUk~lN{xCT(jsw z;Yp@T1SAT~KqTCb3a(cP)&1DNK4h=JduK2)s(9XTV?-4(76KO@#|t*-6s2Dl=w}?& z`IEOv)DCjYr5eow*?MXh8OBgmy;!t>iQ6;NbaZ0pfS*It8}lB6l1(9$AdKemI=5q_ z{7jO%*zS`ZmPK|=YMq+2r-Np*`16dZu!n^|WkFZrL^nhKO "y", "spark.ssl.opt2" -> "z")) + .javaOpts should contain theSameElementsInOrderAs Seq( + "-Dasdf=dfgh", "-Dspark.ssl.opt1=x") + + Worker.maybeUpdateSSLSettings( + cmd("-Dspark.ssl.useNodeLocalConf=false", "-Dspark.ssl.opt1=x"), + conf("spark.ssl.opt1" -> "y", "spark.ssl.opt2" -> "z")) + .javaOpts should contain theSameElementsInOrderAs Seq( + "-Dspark.ssl.useNodeLocalConf=false", "-Dspark.ssl.opt1=x") + + Worker.maybeUpdateSSLSettings( + cmd("-Dspark.ssl.useNodeLocalConf=true", "-Dspark.ssl.opt1=x"), + conf("spark.ssl.opt1" -> "y", "spark.ssl.opt2" -> "z")) + .javaOpts should contain theSameElementsAs Seq( + "-Dspark.ssl.useNodeLocalConf=true", "-Dspark.ssl.opt1=y", "-Dspark.ssl.opt2=z") + + } +} 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 6bbf72e929dcb..39e5d367d676c 100644 --- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.util +import java.util.concurrent.TimeoutException + import scala.concurrent.Await import akka.actor._ @@ -26,6 +28,7 @@ import org.scalatest.FunSuite import org.apache.spark._ import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId +import org.apache.spark.SSLSampleConfigs._ /** @@ -47,7 +50,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val badconf = new SparkConf badconf.set("spark.authenticate", "true") @@ -60,7 +63,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro conf = conf, securityManager = securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( - s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + 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) @@ -74,7 +77,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val conf = new SparkConf conf.set("spark.authenticate", "false") conf.set("spark.authenticate.secret", "bad") - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) val hostname = "localhost" val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, @@ -85,18 +88,18 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val badconf = new SparkConf badconf.set("spark.authenticate", "false") badconf.set("spark.authenticate.secret", "good") - val securityManagerBad = new SecurityManager(badconf); + val securityManagerBad = new SecurityManager(badconf) val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = badconf, securityManager = securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( - s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) val timeout = AkkaUtils.lookupTimeout(conf) slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) @@ -124,7 +127,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val conf = new SparkConf conf.set("spark.authenticate", "true") conf.set("spark.authenticate.secret", "good") - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) val hostname = "localhost" val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, @@ -135,12 +138,12 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val goodconf = new SparkConf goodconf.set("spark.authenticate", "true") goodconf.set("spark.authenticate.secret", "good") - val securityManagerGood = new SecurityManager(goodconf); + val securityManagerGood = new SecurityManager(goodconf) assert(securityManagerGood.isAuthenticationEnabled() === true) @@ -148,7 +151,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro conf = goodconf, securityManager = securityManagerGood) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( - s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) val timeout = AkkaUtils.lookupTimeout(conf) slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) @@ -175,7 +178,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro conf.set("spark.authenticate", "true") conf.set("spark.authenticate.secret", "good") - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) val hostname = "localhost" val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, @@ -186,12 +189,12 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val badconf = new SparkConf badconf.set("spark.authenticate", "false") badconf.set("spark.authenticate.secret", "bad") - val securityManagerBad = new SecurityManager(badconf); + val securityManagerBad = new SecurityManager(badconf) assert(securityManagerBad.isAuthenticationEnabled() === false) @@ -199,7 +202,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro conf = badconf, securityManager = securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( - s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + 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) @@ -209,4 +212,170 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro slaveSystem.shutdown() } + test("remote fetch ssl on") { + val conf = sparkSSLConfig() + 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) + + assert(securityManager.isAuthenticationEnabled() === false) + + val masterTracker = new MapOutputTrackerMaster(conf) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + + val slaveConf = sparkSSLConfig() + val securityManagerBad = new SecurityManager(slaveConf) + + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + conf = slaveConf, securityManager = 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) + + assert(securityManagerBad.isAuthenticationEnabled() === false) + + masterTracker.registerShuffle(10, 1) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) + + val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) + masterTracker.registerMapOutput(10, 0, + MapStatus(BlockManagerId("a", "hostA", 1000), Array(1000L))) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) + + // this should succeed since security off + assert(slaveTracker.getServerStatuses(10, 0).toSeq === + Seq((BlockManagerId("a", "hostA", 1000), size1000))) + + actorSystem.shutdown() + slaveSystem.shutdown() + } + + + test("remote fetch ssl on and security enabled") { + val conf = sparkSSLConfig() + 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) + + assert(securityManager.isAuthenticationEnabled() === true) + + val masterTracker = new MapOutputTrackerMaster(conf) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + + 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 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) + + assert(securityManagerBad.isAuthenticationEnabled() === true) + + masterTracker.registerShuffle(10, 1) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) + + val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) + masterTracker.registerMapOutput(10, 0, + MapStatus(BlockManagerId("a", "hostA", 1000), Array(1000L))) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) + + assert(slaveTracker.getServerStatuses(10, 0).toSeq === + Seq((BlockManagerId("a", "hostA", 1000), size1000))) + + actorSystem.shutdown() + slaveSystem.shutdown() + } + + + test("remote fetch ssl on and security enabled - bad credentials") { + val conf = sparkSSLConfig() + 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) + + assert(securityManager.isAuthenticationEnabled() === true) + + val masterTracker = new MapOutputTrackerMaster(conf) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + + val slaveConf = sparkSSLConfig() + 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 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) + } + + actorSystem.shutdown() + slaveSystem.shutdown() + } + + + test("remote fetch ssl on - untrusted server") { + val conf = sparkSSLConfigUntrusted() + 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) + + assert(securityManager.isAuthenticationEnabled() === false) + + val masterTracker = new MapOutputTrackerMaster(conf) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") + + val slaveConf = sparkSSLConfig() + val securityManagerBad = new SecurityManager(slaveConf) + + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + conf = slaveConf, securityManager = 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[TimeoutException] { + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + } + + actorSystem.shutdown() + slaveSystem.shutdown() + } + } diff --git a/docs/configuration.md b/docs/configuration.md index 08c6befaf31ad..62d3fca937b2c 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1242,6 +1242,86 @@ Apart from these, the following properties are also available, and may be useful +#### Encryption + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaning
spark.ssl.enabledfalse +

Whether to enable SSL connections on all supported protocols.

+ +

All the SSL settings like spark.ssl.xxx where xxx is a + particular configuration property, denote the global configuration for all the supported + protocols. In order to override the global configuration for the particular protocol, + the properties must be overwritten in the protocol-specific namespace.

+ +

Use spark.ssl.YYY.XXX settings to overwrite the global configuration for + particular protocol denoted by YYY. Currently YYY can be + either akka for Akka based connections or fs for broadcast and + file server.

+
spark.ssl.keyStoreNone + A path to a key-store file. The path can be absolute or relative to the directory where + the component is started in. +
spark.ssl.keyStorePasswordNone + A password to the key-store. +
spark.ssl.keyPasswordNone + A password to the private key in key-store. +
spark.ssl.trustStoreNone + A path to a trust-store file. The path can be absolute or relative to the directory + where the component is started in. +
spark.ssl.trustStorePasswordNone + A password to the trust-store. +
spark.ssl.protocolNone + A protocol name. The protocol must be supported by JVM. The reference list of protocols + one can find on this + page. +
spark.ssl.enabledAlgorithmsEmpty + 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 Streaming diff --git a/docs/security.md b/docs/security.md index 1e206a139fb72..6e0a54fbc4ad7 100644 --- a/docs/security.md +++ b/docs/security.md @@ -20,6 +20,30 @@ Spark allows for a set of administrators to be specified in the acls who always If your applications are using event logging, the directory where the event logs go (`spark.eventLog.dir`) should be manually created and have the proper permissions set on it. If you want those log files secured, the permissions should be set to `drwxrwxrwxt` for that directory. The owner of the directory should be the super user who is running the history server and the group permissions should be restricted to super user group. This will allow all users to write to the directory but will prevent unprivileged users from removing or renaming a file unless they own the file or directory. The event log files will be created by Spark with permissions such that only the user and group have read and write access. +## Encryption + +Spark supports SSL for Akka and HTTP (for broadcast and file server) protocols. However SSL is not supported yet for WebUI and block transfer service. + +Connection encryption (SSL) configuration is organized hierarchically. The user can configure the default SSL settings which will be used for all the supported communication protocols unless they are overwritten by protocol-specific settings. This way the user can easily provide the common settings for all the protocols without disabling the ability to configure each one individually. The common SSL settings are at `spark.ssl` namespace in Spark configuration, while Akka SSL configuration is at `spark.ssl.akka` and HTTP for broadcast and file server SSL configuration is at `spark.ssl.fs`. The full breakdown can be found on the [configuration page](configuration.html). + +SSL must be configured on each node and configured for each component involved in communication using the particular protocol. + +### YARN mode +The key-store can be prepared on the client side and then distributed and used by the executors as the part of the application. It is possible because the user is able to deploy files before the application is started in YARN by using `spark.yarn.dist.files` or `spark.yarn.dist.archives` configuration settings. The responsibility for encryption of transferring these files is on YARN side and has nothing to do with Spark. + +### Standalone mode +The user needs to provide key-stores and configuration options for master and workers. They have to be set by attaching appropriate Java system properties in `SPARK_MASTER_OPTS` and in `SPARK_WORKER_OPTS` environment variables, or just in `SPARK_DAEMON_JAVA_OPTS`. In this mode, the user may allow the executors to use the SSL settings inherited from the worker which spawned that executor. It can be accomplished by setting `spark.ssl.useNodeLocalConf` to `true`. If that parameter is set, the settings provided by user on the client side, are not used by the executors. + +### Preparing the key-stores +Key-stores can be generated by `keytool` program. The reference documentation for this tool is +[here](https://docs.oracle.com/javase/7/docs/technotes/tools/solaris/keytool.html). The most basic +steps to configure the key-stores and the trust-store for the standalone deployment mode is as +follows: +* Generate a keys pair for each node +* Export the public key of the key pair to a file on each node +* Import all exported public keys into a single trust-store +* Distribute the trust-store over the nodes + ## Configuring Ports for Network Security Spark makes heavy use of the network, and some environments have strict requirements for using tight 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 b46df12da86dc..9805609120005 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -45,7 +45,7 @@ class ExecutorClassLoader(conf: SparkConf, classUri: String, parent: ClassLoader // Hadoop FileSystem object for our URI, if it isn't using HTTP var fileSystem: FileSystem = { - if (uri.getScheme() == "http") { + if (Set("http", "https", "ftp").contains(uri.getScheme)) { null } else { FileSystem.get(uri, SparkHadoopUtil.get.newConfiguration(conf)) @@ -78,13 +78,16 @@ class ExecutorClassLoader(conf: SparkConf, classUri: String, parent: ClassLoader if (fileSystem != null) { fileSystem.open(new Path(directory, pathInDirectory)) } else { - if (SparkEnv.get.securityManager.isAuthenticationEnabled()) { + val url = if (SparkEnv.get.securityManager.isAuthenticationEnabled()) { val uri = new URI(classUri + "/" + urlEncode(pathInDirectory)) val newuri = Utils.constructURIForAuthentication(uri, SparkEnv.get.securityManager) - newuri.toURL().openStream() + newuri.toURL } else { - new URL(classUri + "/" + urlEncode(pathInDirectory)).openStream() + new URL(classUri + "/" + urlEncode(pathInDirectory)) } + + Utils.setupSecureURLConnection(url.openConnection(), SparkEnv.get.securityManager) + .getInputStream } } val bytes = readAndTransformClass(name, inputStream) 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 716cf2c7f32fc..7d29ed88cfcb4 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 @@ -67,8 +67,12 @@ private[streaming] class ReceiverSupervisorImpl( private val trackerActor = { val ip = env.conf.get("spark.driver.host", "localhost") val port = env.conf.getInt("spark.driver.port", 7077) - val url = "akka.tcp://%s@%s:%s/user/ReceiverTracker".format( - SparkEnv.driverActorSystemName, ip, port) + val url = AkkaUtils.address( + AkkaUtils.protocol(env.actorSystem), + SparkEnv.driverActorSystemName, + ip, + port, + "ReceiverTracker") env.actorSystem.actorSelection(url) } 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 eb328b2b8ac50..37e98e01fddf7 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 @@ -244,7 +244,9 @@ private[spark] class ApplicationMaster( host: String, port: String, isDriver: Boolean): Unit = { - val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + + val driverUrl = AkkaUtils.address( + AkkaUtils.protocol(actorSystem), SparkEnv.driverActorSystemName, host, port, 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 040406c15030e..0dbb6154b3039 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 @@ -34,9 +34,10 @@ import org.apache.hadoop.yarn.util.RackResolver import org.apache.log4j.{Level, Logger} -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SparkEnv, Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.util.AkkaUtils /** * YarnAllocator is charged with requesting containers from the YARN ResourceManager and deciding @@ -106,7 +107,9 @@ private[yarn] class YarnAllocator( new ThreadFactoryBuilder().setNameFormat("ContainerLauncher #%d").setDaemon(true).build()) launcherPool.allowCoreThreadTimeOut(true) - private val driverUrl = "akka.tcp://sparkDriver@%s:%s/user/%s".format( + private val driverUrl = AkkaUtils.address( + AkkaUtils.protocol(securityMgr.akkaSSLOptions.enabled), + SparkEnv.driverActorSystemName, sparkConf.get("spark.driver.host"), sparkConf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) From eccb9fbb2d1bf6f7c65fb4f017e9205bb3034ec6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 2 Feb 2015 17:52:17 -0800 Subject: [PATCH 022/578] Revert "[SPARK-4508] [SQL] build native date type to conform behavior to Hive" This reverts commit 1646f89d967913ee1f231d9606f8502d13c25804. --- .../main/scala/org/apache/spark/sql/Row.scala | 2 +- .../spark/sql/catalyst/ScalaReflection.scala | 9 ++- .../apache/spark/sql/catalyst/SqlParser.scala | 2 - .../spark/sql/catalyst/expressions/Cast.scala | 53 +++++++++------- .../expressions/codegen/CodeGenerator.scala | 3 - .../sql/catalyst/expressions/literals.scala | 2 +- .../apache/spark/sql/types/DateUtils.scala | 60 ------------------- .../apache/spark/sql/types/dataTypes.scala | 12 ++-- .../ExpressionEvaluationSuite.scala | 28 ++++----- .../spark/sql/types/DataTypeSuite.scala | 2 +- .../spark/sql/columnar/ColumnStats.scala | 19 +++++- .../spark/sql/columnar/ColumnType.scala | 13 ++-- .../spark/sql/execution/pythonUdfs.scala | 4 +- .../org/apache/spark/sql/json/JsonRDD.scala | 6 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 7 --- .../sql/ScalaReflectionRelationSuite.scala | 3 +- .../spark/sql/columnar/ColumnStatsSuite.scala | 2 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 6 +- .../sql/columnar/ColumnarTestUtils.scala | 6 +- .../org/apache/spark/sql/json/JsonSuite.scala | 7 +-- .../execution/HiveCompatibilitySuite.scala | 1 - .../apache/spark/sql/hive/HiveContext.scala | 4 +- .../spark/sql/hive/HiveInspectors.scala | 11 +--- .../apache/spark/sql/hive/TableReader.scala | 3 +- ...te cast-0-a7cd69b80c77a771a2c955db666be53d | 1 - ... test 1-0-bde89be08a12361073ff658fef768b7e | 1 - ... test 2-0-dc1b267f1d79d49e6675afe4fd2a34a5 | 1 - ...date_1-0-23edf29bf7376c70d5ecf12720f4b1eb} | 0 .../date_1-0-50131c0ba7b7a6b65c789a5a8497bada | 1 - ...date_1-1-4ebe3571c13a8b0c03096fbd972b7f1b} | 0 ...ate_1-10-d964bec7e5632091ab5cb6f6786dbbf9} | 0 ...ate_1-11-480c5f024a28232b7857be327c992509} | 0 ...ate_1-12-4c0ed7fcb75770d8790575b586bf14f4} | 0 ...date_1-13-44fc74c1993062c0a9522199ff27fea} | 0 ...ate_1-14-4855a66124b16d1d0d003235995ac06b} | 0 ...ate_1-15-8bc190dba0f641840b5e1e198a14c55b} | 0 ...ate_1-16-23edf29bf7376c70d5ecf12720f4b1eb} | 0 ...date_1-2-abdce0c0d14d3fc7441b7c134b02f99a} | 0 ...date_1-3-df16364a220ff96a6ea1cd478cbc1d0b} | 0 ...date_1-4-d964bec7e5632091ab5cb6f6786dbbf9} | 0 ...date_1-5-5e70fc74158fbfca38134174360de12d} | 0 ...date_1-6-df16364a220ff96a6ea1cd478cbc1d0b} | 0 ...date_1-7-d964bec7e5632091ab5cb6f6786dbbf9} | 0 ...date_1-8-1d5c58095cd52ea539d869f2ab1ab67d} | 0 ...date_1-9-df16364a220ff96a6ea1cd478cbc1d0b} | 0 .../spark/sql/hive/HiveInspectorSuite.scala | 4 +- .../sql/hive/execution/HiveQuerySuite.scala | 26 +------- .../org/apache/spark/sql/hive/Shim12.scala | 2 +- .../org/apache/spark/sql/hive/Shim13.scala | 2 +- 49 files changed, 112 insertions(+), 191 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala delete mode 100644 sql/hive/src/test/resources/golden/Date cast-0-a7cd69b80c77a771a2c955db666be53d delete mode 100644 sql/hive/src/test/resources/golden/Date comparison test 1-0-bde89be08a12361073ff658fef768b7e delete mode 100644 sql/hive/src/test/resources/golden/Date comparison test 2-0-dc1b267f1d79d49e6675afe4fd2a34a5 rename sql/hive/src/test/resources/golden/{date_1-1-23edf29bf7376c70d5ecf12720f4b1eb => date_1-0-23edf29bf7376c70d5ecf12720f4b1eb} (100%) delete mode 100644 sql/hive/src/test/resources/golden/date_1-0-50131c0ba7b7a6b65c789a5a8497bada rename sql/hive/src/test/resources/golden/{date_1-17-23edf29bf7376c70d5ecf12720f4b1eb => date_1-1-4ebe3571c13a8b0c03096fbd972b7f1b} (100%) rename sql/hive/src/test/resources/golden/{date_1-11-d964bec7e5632091ab5cb6f6786dbbf9 => date_1-10-d964bec7e5632091ab5cb6f6786dbbf9} (100%) rename sql/hive/src/test/resources/golden/{date_1-12-480c5f024a28232b7857be327c992509 => date_1-11-480c5f024a28232b7857be327c992509} (100%) rename sql/hive/src/test/resources/golden/{date_1-13-4c0ed7fcb75770d8790575b586bf14f4 => date_1-12-4c0ed7fcb75770d8790575b586bf14f4} (100%) rename sql/hive/src/test/resources/golden/{date_1-14-44fc74c1993062c0a9522199ff27fea => date_1-13-44fc74c1993062c0a9522199ff27fea} (100%) rename sql/hive/src/test/resources/golden/{date_1-15-4855a66124b16d1d0d003235995ac06b => date_1-14-4855a66124b16d1d0d003235995ac06b} (100%) rename sql/hive/src/test/resources/golden/{date_1-16-8bc190dba0f641840b5e1e198a14c55b => date_1-15-8bc190dba0f641840b5e1e198a14c55b} (100%) rename sql/hive/src/test/resources/golden/{date_1-2-4ebe3571c13a8b0c03096fbd972b7f1b => date_1-16-23edf29bf7376c70d5ecf12720f4b1eb} (100%) rename sql/hive/src/test/resources/golden/{date_1-3-26b5c291400dfde455b3c1b878b71d0 => date_1-2-abdce0c0d14d3fc7441b7c134b02f99a} (100%) rename sql/hive/src/test/resources/golden/{date_1-10-df16364a220ff96a6ea1cd478cbc1d0b => date_1-3-df16364a220ff96a6ea1cd478cbc1d0b} (100%) rename sql/hive/src/test/resources/golden/{date_1-5-d964bec7e5632091ab5cb6f6786dbbf9 => date_1-4-d964bec7e5632091ab5cb6f6786dbbf9} (100%) rename sql/hive/src/test/resources/golden/{date_1-6-559d01fb0b42c42f0c4927fa0f9deac4 => date_1-5-5e70fc74158fbfca38134174360de12d} (100%) rename sql/hive/src/test/resources/golden/{date_1-4-df16364a220ff96a6ea1cd478cbc1d0b => date_1-6-df16364a220ff96a6ea1cd478cbc1d0b} (100%) rename sql/hive/src/test/resources/golden/{date_1-8-d964bec7e5632091ab5cb6f6786dbbf9 => date_1-7-d964bec7e5632091ab5cb6f6786dbbf9} (100%) rename sql/hive/src/test/resources/golden/{date_1-9-8306558e0eabe936ac33dabaaa17fea4 => date_1-8-1d5c58095cd52ea539d869f2ab1ab67d} (100%) rename sql/hive/src/test/resources/golden/{date_1-7-df16364a220ff96a6ea1cd478cbc1d0b => date_1-9-df16364a220ff96a6ea1cd478cbc1d0b} (100%) 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..41bb4f012f2e1 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 + object Row { /** 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 8e79e532ca564..e0db587efb08d 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 @@ -17,13 +17,14 @@ package org.apache.spark.sql.catalyst -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import org.apache.spark.util.Utils import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute, AttributeReference, Row} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.types._ + /** * A default version of ScalaReflection that uses the runtime universe. */ @@ -71,7 +72,6 @@ trait ScalaReflection { }.toArray) case (d: BigDecimal, _) => Decimal(d) case (d: java.math.BigDecimal, _) => Decimal(d) - case (d: java.sql.Date, _) => DateUtils.fromJavaDate(d) case (other, _) => other } @@ -85,7 +85,6 @@ trait ScalaReflection { } case (r: Row, s: StructType) => convertRowToScala(r, s) case (d: Decimal, _: DecimalType) => d.toJavaBigDecimal - case (i: Int, DateType) => DateUtils.toJavaDate(i) case (other, _) => other } @@ -160,7 +159,7 @@ trait ScalaReflection { 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) + case t if t <:< typeOf[Date] => Schema(DateType, nullable = true) case t if t <:< typeOf[BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) case t if t <:< typeOf[java.math.BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) case t if t <:< typeOf[Decimal] => Schema(DecimalType.Unlimited, nullable = true) @@ -192,7 +191,7 @@ trait ScalaReflection { case obj: LongType.JvmType => LongType case obj: FloatType.JvmType => FloatType case obj: DoubleType.JvmType => DoubleType - case obj: java.sql.Date => DateType + case obj: DateType.JvmType => DateType case obj: java.math.BigDecimal => DecimalType.Unlimited case obj: Decimal => DecimalType.Unlimited case obj: TimestampType.JvmType => TimestampType 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 2ce8be8e24e85..594a423146d77 100755 --- 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 @@ -52,7 +52,6 @@ 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") @@ -384,7 +383,6 @@ class SqlParser extends AbstractSparkSQLParser { | DOUBLE ^^^ DoubleType | fixedDecimalType | DECIMAL ^^^ DecimalType.Unlimited - | DATE ^^^ DateType ) protected lazy val fixedDecimalType: Parser[DataType] = 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..ece5ee73618cb 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 @@ -113,7 +113,7 @@ 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 DateType => buildCast[Date](_, dateToString) case TimestampType => buildCast[Timestamp](_, timestampToString) case _ => buildCast[Any](_, _.toString) } @@ -131,7 +131,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w buildCast[Timestamp](_, t => t.getTime() != 0 || t.getNanos() != 0) case DateType => // Hive would return null when cast from date to boolean - buildCast[Int](_, d => null) + buildCast[Date](_, d => null) case LongType => buildCast[Long](_, _ != 0) case IntegerType => @@ -171,7 +171,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case ByteType => buildCast[Byte](_, b => new Timestamp(b)) case DateType => - buildCast[Int](_, d => new Timestamp(DateUtils.toJavaDate(d).getTime)) + buildCast[Date](_, d => new Timestamp(d.getTime)) // TimestampWritable.decimalToTimestamp case DecimalType() => buildCast[Decimal](_, d => decimalToTimestamp(d)) @@ -224,24 +224,37 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w } } + // Converts Timestamp to string according to Hive TimestampWritable convention + private[this] def timestampToDateString(ts: Timestamp): String = { + Cast.threadLocalDateFormat.get.format(ts) + } + // DateConverter private[this] def castToDate(from: DataType): Any => Any = from match { case StringType => buildCast[String](_, s => - try DateUtils.fromJavaDate(Date.valueOf(s)) - catch { case _: java.lang.IllegalArgumentException => null } - ) + try Date.valueOf(s) catch { case _: java.lang.IllegalArgumentException => null }) case TimestampType => // throw valid precision more than seconds, according to Hive. // Timestamp.nanos is in 0 to 999,999,999, no more than a second. - buildCast[Timestamp](_, t => DateUtils.millisToDays(t.getTime)) + buildCast[Timestamp](_, t => new Date(Math.floor(t.getTime / 1000.0).toLong * 1000)) // Hive throws this exception as a Semantic Exception - // It is never possible to compare result when hive return with exception, - // so we can return null + // It is never possible to compare result when hive return with exception, so we can return null // NULL is more reasonable here, since the query itself obeys the grammar. case _ => _ => null } + // Date cannot be cast to long, according to hive + private[this] def dateToLong(d: Date) = null + + // Date cannot be cast to double, according to hive + private[this] def dateToDouble(d: Date) = null + + // Converts Date to string according to Hive DateWritable convention + private[this] def dateToString(d: Date): String = { + Cast.threadLocalDateFormat.get.format(d) + } + // LongConverter private[this] def castToLong(from: DataType): Any => Any = from match { case StringType => @@ -251,7 +264,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => if (b) 1L else 0L) case DateType => - buildCast[Int](_, d => null) + buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t)) case x: NumericType => @@ -267,7 +280,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => if (b) 1 else 0) case DateType => - buildCast[Int](_, d => null) + buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toInt) case x: NumericType => @@ -283,7 +296,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => if (b) 1.toShort else 0.toShort) case DateType => - buildCast[Int](_, d => null) + buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toShort) case x: NumericType => @@ -299,7 +312,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => if (b) 1.toByte else 0.toByte) case DateType => - buildCast[Int](_, d => null) + buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toByte) case x: NumericType => @@ -329,7 +342,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => changePrecision(if (b) Decimal(1) else Decimal(0), target)) case DateType => - buildCast[Int](_, d => null) // date can't cast to decimal in Hive + buildCast[Date](_, d => null) // date can't cast to decimal in Hive case TimestampType => // Note that we lose precision here. buildCast[Timestamp](_, t => changePrecision(Decimal(timestampToDouble(t)), target)) @@ -354,7 +367,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => if (b) 1d else 0d) case DateType => - buildCast[Int](_, d => null) + buildCast[Date](_, d => dateToDouble(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToDouble(t)) case x: NumericType => @@ -370,7 +383,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => if (b) 1f else 0f) case DateType => - buildCast[Int](_, d => null) + buildCast[Date](_, d => dateToDouble(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToDouble(t).toFloat) case x: NumericType => @@ -429,16 +442,16 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w object Cast { // `SimpleDateFormat` is not thread-safe. - private[sql] val threadLocalTimestampFormat = new ThreadLocal[DateFormat] { + private[sql] val threadLocalDateFormat = new ThreadLocal[DateFormat] { override def initialValue() = { - new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + new SimpleDateFormat("yyyy-MM-dd") } } // `SimpleDateFormat` is not thread-safe. - private[sql] val threadLocalDateFormat = new ThreadLocal[DateFormat] { + private[sql] val threadLocalTimestampFormat = new ThreadLocal[DateFormat] { override def initialValue() = { - new SimpleDateFormat("yyyy-MM-dd") + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") } } } 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..4cae5c4718683 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 @@ -246,9 +246,6 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin new String(${eval.primitiveTerm}.asInstanceOf[Array[Byte]]) """.children - case Cast(child @ DateType(), StringType) => - child.castOrNull(c => q"org.apache.spark.sql.types.DateUtils.toString($c)", StringType) - case Cast(child @ NumericType(), IntegerType) => child.castOrNull(c => q"$c.toInt", IntegerType) 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..5b389aad7a85d 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 @@ -35,7 +35,7 @@ object Literal { case d: java.math.BigDecimal => Literal(Decimal(d), DecimalType.Unlimited) case d: Decimal => Literal(d, DecimalType.Unlimited) case t: Timestamp => Literal(t, TimestampType) - case d: Date => Literal(DateUtils.fromJavaDate(d), DateType) + case d: Date => Literal(d, DateType) case a: Array[Byte] => Literal(a, BinaryType) case null => Literal(null, NullType) } 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 deleted file mode 100644 index 8a1a3b81b3d2c..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala +++ /dev/null @@ -1,60 +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.sql.Date -import java.util.{Calendar, TimeZone} - -import org.apache.spark.sql.catalyst.expressions.Cast - -/** - * helper function to convert between Int value of days since 1970-01-01 and java.sql.Date - */ -object DateUtils { - private val MILLIS_PER_DAY = 86400000 - - // Java TimeZone has no mention of thread safety. Use thread local instance to be safe. - private val LOCAL_TIMEZONE = new ThreadLocal[TimeZone] { - override protected def initialValue: TimeZone = { - Calendar.getInstance.getTimeZone - } - } - - private def javaDateToDays(d: Date): Int = { - millisToDays(d.getTime) - } - - def millisToDays(millisLocal: Long): Int = { - ((millisLocal + LOCAL_TIMEZONE.get().getOffset(millisLocal)) / MILLIS_PER_DAY).toInt - } - - private def toMillisSinceEpoch(days: Int): Long = { - val millisUtc = days.toLong * MILLIS_PER_DAY - millisUtc - LOCAL_TIMEZONE.get().getOffset(millisUtc) - } - - def fromJavaDate(date: java.sql.Date): Int = { - javaDateToDays(date) - } - - def toJavaDate(daysSinceEpoch: Int): java.sql.Date = { - new java.sql.Date(toMillisSinceEpoch(daysSinceEpoch)) - } - - def toString(days: Int): String = Cast.threadLocalDateFormat.get.format(toJavaDate(days)) -} 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 8ca0769fac287..6ab99aa38877f 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.types -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import scala.math.Numeric.{FloatAsIfIntegral, DoubleAsIfIntegral} import scala.reflect.ClassTag @@ -387,16 +387,18 @@ case object TimestampType extends NativeType { */ @DeveloperApi case object DateType extends NativeType { - private[sql] type JvmType = Int + private[sql] type JvmType = Date @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - private[sql] val ordering = implicitly[Ordering[JvmType]] + private[sql] val ordering = new Ordering[JvmType] { + def compare(x: Date, y: Date) = x.compareTo(y) + } /** - * The default size of a value of the DateType is 4 bytes. + * The default size of a value of the DateType is 8 bytes. */ - override def defaultSize: Int = 4 + override def defaultSize: Int = 8 } 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 25d1c105a00a6..37e64adeea853 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 @@ -303,7 +303,6 @@ class ExpressionEvaluationSuite extends FunSuite { val sd = "1970-01-01" val d = Date.valueOf(sd) - val zts = sd + " 00:00:00" val sts = sd + " 00:00:02" val nts = sts + ".1" val ts = Timestamp.valueOf(nts) @@ -320,14 +319,14 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble) checkEvaluation(Cast(Literal(sd) cast DateType, StringType), sd) - checkEvaluation(Cast(Literal(d) cast StringType, DateType), 0) + checkEvaluation(Cast(Literal(d) cast StringType, DateType), d) checkEvaluation(Cast(Literal(nts) cast TimestampType, StringType), nts) checkEvaluation(Cast(Literal(ts) cast StringType, TimestampType), ts) // all convert to string type to check checkEvaluation( Cast(Cast(Literal(nts) cast TimestampType, DateType), StringType), sd) checkEvaluation( - Cast(Cast(Literal(ts) cast DateType, TimestampType), StringType), zts) + Cast(Cast(Literal(ts) cast DateType, TimestampType), StringType), sts) checkEvaluation(Cast("abdef" cast BinaryType, StringType), "abdef") @@ -378,8 +377,8 @@ class ExpressionEvaluationSuite extends FunSuite { } test("date") { - val d1 = DateUtils.fromJavaDate(Date.valueOf("1970-01-01")) - val d2 = DateUtils.fromJavaDate(Date.valueOf("1970-01-02")) + val d1 = Date.valueOf("1970-01-01") + val d2 = Date.valueOf("1970-01-02") checkEvaluation(Literal(d1) < Literal(d2), true) } @@ -460,21 +459,22 @@ class ExpressionEvaluationSuite extends FunSuite { test("date casting") { val d = Date.valueOf("1970-01-01") - checkEvaluation(Cast(Literal(d), ShortType), null) - checkEvaluation(Cast(Literal(d), IntegerType), null) - checkEvaluation(Cast(Literal(d), LongType), null) - checkEvaluation(Cast(Literal(d), FloatType), null) - checkEvaluation(Cast(Literal(d), DoubleType), null) - checkEvaluation(Cast(Literal(d), DecimalType.Unlimited), null) - checkEvaluation(Cast(Literal(d), DecimalType(10, 2)), null) - checkEvaluation(Cast(Literal(d), StringType), "1970-01-01") - checkEvaluation(Cast(Cast(Literal(d), TimestampType), StringType), "1970-01-01 00:00:00") + checkEvaluation(Cast(d, ShortType), null) + checkEvaluation(Cast(d, IntegerType), null) + checkEvaluation(Cast(d, LongType), null) + checkEvaluation(Cast(d, FloatType), null) + checkEvaluation(Cast(d, DoubleType), null) + checkEvaluation(Cast(d, DecimalType.Unlimited), null) + checkEvaluation(Cast(d, DecimalType(10, 2)), null) + checkEvaluation(Cast(d, StringType), "1970-01-01") + checkEvaluation(Cast(Cast(d, TimestampType), StringType), "1970-01-01 00:00:00") } test("timestamp casting") { val millis = 15 * 1000 + 2 val seconds = millis * 1000 + 2 val ts = new Timestamp(millis) + val ts1 = new Timestamp(15 * 1000) // a timestamp without the milliseconds part val tss = new Timestamp(seconds) checkEvaluation(Cast(ts, ShortType), 15) checkEvaluation(Cast(ts, IntegerType), 15) 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 7bcd6687d11a1..c147be9f6b1ae 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 @@ -106,7 +106,7 @@ class DataTypeSuite extends FunSuite { checkDefaultSize(DoubleType, 8) checkDefaultSize(DecimalType(10, 5), 4096) checkDefaultSize(DecimalType.Unlimited, 4096) - checkDefaultSize(DateType, 4) + checkDefaultSize(DateType, 8) checkDefaultSize(TimestampType, 8) checkDefaultSize(StringType, 4096) checkDefaultSize(BinaryType, 4096) 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..391b3dae5c8ce 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.columnar -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.{AttributeMap, Attribute, AttributeReference} @@ -215,7 +215,22 @@ private[sql] class StringColumnStats extends ColumnStats { def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } -private[sql] class DateColumnStats extends IntColumnStats +private[sql] class DateColumnStats extends ColumnStats { + protected var upper: Date = null + protected var lower: Date = null + + override def gatherStats(row: Row, ordinal: Int) { + super.gatherStats(row, ordinal) + if (!row.isNullAt(ordinal)) { + val value = row(ordinal).asInstanceOf[Date] + if (upper == null || value.compareTo(upper) > 0) upper = value + if (lower == null || value.compareTo(lower) < 0) lower = value + sizeInBytes += DATE.defaultSize + } + } + + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) +} private[sql] class TimestampColumnStats extends ColumnStats { protected var upper: Timestamp = null 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..fcf2faa0914c0 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 @@ -335,20 +335,21 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { } } -private[sql] object DATE extends NativeColumnType(DateType, 8, 4) { +private[sql] object DATE extends NativeColumnType(DateType, 8, 8) { override def extract(buffer: ByteBuffer) = { - buffer.getInt + val date = new Date(buffer.getLong()) + date } - override def append(v: Int, buffer: ByteBuffer): Unit = { - buffer.putInt(v) + override def append(v: Date, buffer: ByteBuffer): Unit = { + buffer.putLong(v.getTime) } override def getField(row: Row, ordinal: Int) = { - row(ordinal).asInstanceOf[Int] + row(ordinal).asInstanceOf[Date] } - def setField(row: MutableRow, ordinal: Int, value: Int): Unit = { + override def setField(row: MutableRow, ordinal: Int, value: Date): Unit = { row(ordinal) = value } } 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..b85021acc9d4c 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 @@ -135,8 +135,6 @@ object EvaluatePython { case (ud, udt: UserDefinedType[_]) => toJava(udt.serialize(ud), udt.sqlType) - case (date: Int, DateType) => DateUtils.toJavaDate(date) - // Pyrolite can handle Timestamp and Decimal case (other, _) => other } @@ -173,7 +171,7 @@ object EvaluatePython { }): Row case (c: java.util.Calendar, DateType) => - DateUtils.fromJavaDate(new java.sql.Date(c.getTime().getTime())) + new java.sql.Date(c.getTime().getTime()) case (c: java.util.Calendar, TimestampType) => new java.sql.Timestamp(c.getTime().getTime()) 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..9171939f7e8f7 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 @@ -377,12 +377,10 @@ private[sql] object JsonRDD extends Logging { } } - private def toDate(value: Any): Int = { + private def toDate(value: Any): Date = { value match { // only support string as date - case value: java.lang.String => - DateUtils.millisToDays(DataTypeConversions.stringToTime(value).getTime) - case value: java.sql.Date => DateUtils.fromJavaDate(value) + case value: java.lang.String => new Date(DataTypeConversions.stringToTime(value).getTime) } } 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 a7f6a50a04fbd..d82c34316cefa 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 @@ -296,13 +296,6 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { mapData.collect().take(1).map(Row.fromTuple).toSeq) } - test("date row") { - checkAnswer(sql( - """select cast("2015-01-28" as date) from testData limit 1"""), - Row(java.sql.Date.valueOf("2015-01-28")) - ) - } - 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"), 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 f26fcc0385b68..a015884bae282 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 @@ -83,8 +83,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 Timestamp(12345), Seq(1,2,3))) + new java.math.BigDecimal(1), new Date(12345), new Timestamp(12345), Seq(1,2,3))) } test("query case class RDD with nulls") { 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..be2b34de077c9 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 @@ -30,7 +30,7 @@ class ColumnStatsSuite extends FunSuite { testColumnStats(classOf[FloatColumnStats], FLOAT, Row(Float.MaxValue, Float.MinValue, 0)) testColumnStats(classOf[DoubleColumnStats], DOUBLE, Row(Double.MaxValue, Double.MinValue, 0)) testColumnStats(classOf[StringColumnStats], STRING, Row(null, null, 0)) - testColumnStats(classOf[DateColumnStats], DATE, Row(Int.MaxValue, Int.MinValue, 0)) + testColumnStats(classOf[DateColumnStats], DATE, Row(null, null, 0)) testColumnStats(classOf[TimestampColumnStats], TIMESTAMP, Row(null, null, 0)) def testColumnStats[T <: NativeType, U <: ColumnStats]( 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..87e608a8853dc 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import org.scalatest.FunSuite @@ -34,7 +34,7 @@ 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) + STRING -> 8, DATE -> 8, TIMESTAMP -> 12, BINARY -> 16, GENERIC -> 16) checks.foreach { case (columnType, expectedSize) => assertResult(expectedSize, s"Wrong defaultSize for $columnType") { @@ -64,7 +64,7 @@ class ColumnTypeSuite extends FunSuite with Logging { checkActualSize(FLOAT, Float.MaxValue, 4) checkActualSize(BOOLEAN, true, 1) checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) - checkActualSize(DATE, 0, 4) + checkActualSize(DATE, new Date(0L), 8) checkActualSize(TIMESTAMP, new Timestamp(0L), 12) val binary = Array.fill[Byte](4)(0: 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 60ed28cc97bf1..f941465fa3e35 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 @@ -17,11 +17,11 @@ package org.apache.spark.sql.columnar -import java.sql.Timestamp - import scala.collection.immutable.HashSet import scala.util.Random +import java.sql.{Date, Timestamp} + import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.types.{DataType, NativeType} @@ -50,7 +50,7 @@ object ColumnarTestUtils { case STRING => Random.nextString(Random.nextInt(32)) case BOOLEAN => Random.nextBoolean() case BINARY => randomBytes(Random.nextInt(32)) - case DATE => Random.nextInt() + case DATE => new Date(Random.nextLong()) 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/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 1396c6b7246d1..cb615388da0c7 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 @@ -67,15 +67,14 @@ class JsonSuite extends QueryTest { checkTypePromotion(Timestamp.valueOf(strTime), enforceCorrectType(strTime, TimestampType)) val strDate = "2014-10-15" - checkTypePromotion( - DateUtils.fromJavaDate(Date.valueOf(strDate)), enforceCorrectType(strDate, DateType)) + checkTypePromotion(Date.valueOf(strDate), enforceCorrectType(strDate, DateType)) val ISO8601Time1 = "1970-01-01T01:00:01.0Z" checkTypePromotion(new Timestamp(3601000), enforceCorrectType(ISO8601Time1, TimestampType)) - checkTypePromotion(DateUtils.millisToDays(3601000), enforceCorrectType(ISO8601Time1, DateType)) + checkTypePromotion(new Date(3601000), enforceCorrectType(ISO8601Time1, DateType)) val ISO8601Time2 = "1970-01-01T02:00:01-01:00" checkTypePromotion(new Timestamp(10801000), enforceCorrectType(ISO8601Time2, TimestampType)) - checkTypePromotion(DateUtils.millisToDays(10801000), enforceCorrectType(ISO8601Time2, DateType)) + checkTypePromotion(new Date(10801000), enforceCorrectType(ISO8601Time2, DateType)) } test("Get compatible type") { 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..0d934620aca09 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 @@ -357,7 +357,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "database_drop", "database_location", "database_properties", - "date_1", "date_2", "date_3", "date_4", 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 724bd28d4b608..b746942cb1067 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive import java.io.{BufferedReader, InputStreamReader, PrintStream} -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import scala.collection.JavaConversions._ import scala.language.implicitConversions @@ -409,7 +409,7 @@ private object HiveContext { toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) }.toSeq.sorted.mkString("{", ",", "}") case (null, _) => "NULL" - case (d: Int, DateType) => new DateWritable(d).toString + case (d: Date, DateType) => new DateWritable(d).toString case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8") case (decimal: java.math.BigDecimal, DecimalType()) => 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..82dba99900df9 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 @@ -267,8 +267,7 @@ private[hive] trait HiveInspectors { val temp = new Array[Byte](writable.getLength) System.arraycopy(writable.getBytes, 0, temp, 0, temp.length) temp - case poi: WritableConstantDateObjectInspector => - DateUtils.fromJavaDate(poi.getWritableConstantValue.get()) + case poi: WritableConstantDateObjectInspector => poi.getWritableConstantValue.get() case mi: StandardConstantMapObjectInspector => // take the value from the map inspector object, rather than the input data mi.getWritableConstantValue.map { case (k, v) => @@ -305,8 +304,7 @@ private[hive] trait HiveInspectors { System.arraycopy(bw.getBytes(), 0, result, 0, bw.getLength()) result case x: DateObjectInspector if x.preferWritable() => - DateUtils.fromJavaDate(x.getPrimitiveWritableObject(data).get()) - case x: DateObjectInspector => DateUtils.fromJavaDate(x.getPrimitiveJavaObject(data)) + x.getPrimitiveWritableObject(data).get() // org.apache.hadoop.hive.serde2.io.TimestampWritable.set will reset current time object // if next timestamp is null, so Timestamp object is cloned case x: TimestampObjectInspector if x.preferWritable() => @@ -345,9 +343,6 @@ private[hive] trait HiveInspectors { case _: JavaHiveDecimalObjectInspector => (o: Any) => HiveShim.createDecimal(o.asInstanceOf[Decimal].toJavaBigDecimal) - case _: JavaDateObjectInspector => - (o: Any) => DateUtils.toJavaDate(o.asInstanceOf[Int]) - case soi: StandardStructObjectInspector => val wrappers = soi.getAllStructFieldRefs.map(ref => wrapperFor(ref.getFieldObjectInspector)) (o: Any) => { @@ -431,7 +426,7 @@ private[hive] trait HiveInspectors { case _: BinaryObjectInspector if x.preferWritable() => HiveShim.getBinaryWritable(a) case _: BinaryObjectInspector => a.asInstanceOf[Array[Byte]] case _: DateObjectInspector if x.preferWritable() => HiveShim.getDateWritable(a) - case _: DateObjectInspector => DateUtils.toJavaDate(a.asInstanceOf[Int]) + case _: DateObjectInspector => a.asInstanceOf[java.sql.Date] case _: TimestampObjectInspector if x.preferWritable() => HiveShim.getTimestampWritable(a) case _: TimestampObjectInspector => a.asInstanceOf[java.sql.Timestamp] } 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..c368715f7c6f5 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 @@ -34,7 +34,6 @@ import org.apache.spark.SerializableWritable 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 /** * A trait for subclasses that handle table scans. @@ -307,7 +306,7 @@ private[hive] object HadoopTableReader extends HiveInspectors { row.update(ordinal, oi.getPrimitiveJavaObject(value).clone()) case oi: DateObjectInspector => (value: Any, row: MutableRow, ordinal: Int) => - row.update(ordinal, DateUtils.fromJavaDate(oi.getPrimitiveJavaObject(value))) + row.update(ordinal, oi.getPrimitiveJavaObject(value)) case oi: BinaryObjectInspector => (value: Any, row: MutableRow, ordinal: Int) => row.update(ordinal, oi.getPrimitiveJavaObject(value)) diff --git a/sql/hive/src/test/resources/golden/Date cast-0-a7cd69b80c77a771a2c955db666be53d b/sql/hive/src/test/resources/golden/Date cast-0-a7cd69b80c77a771a2c955db666be53d deleted file mode 100644 index 98da82fa89386..0000000000000 --- a/sql/hive/src/test/resources/golden/Date cast-0-a7cd69b80c77a771a2c955db666be53d +++ /dev/null @@ -1 +0,0 @@ -1970-01-01 1970-01-01 1969-12-31 16:00:00 1969-12-31 16:00:00 1970-01-01 00:00:00 diff --git a/sql/hive/src/test/resources/golden/Date comparison test 1-0-bde89be08a12361073ff658fef768b7e b/sql/hive/src/test/resources/golden/Date comparison test 1-0-bde89be08a12361073ff658fef768b7e deleted file mode 100644 index 27ba77ddaf615..0000000000000 --- a/sql/hive/src/test/resources/golden/Date comparison test 1-0-bde89be08a12361073ff658fef768b7e +++ /dev/null @@ -1 +0,0 @@ -true diff --git a/sql/hive/src/test/resources/golden/Date comparison test 2-0-dc1b267f1d79d49e6675afe4fd2a34a5 b/sql/hive/src/test/resources/golden/Date comparison test 2-0-dc1b267f1d79d49e6675afe4fd2a34a5 deleted file mode 100644 index 27ba77ddaf615..0000000000000 --- a/sql/hive/src/test/resources/golden/Date comparison test 2-0-dc1b267f1d79d49e6675afe4fd2a34a5 +++ /dev/null @@ -1 +0,0 @@ -true diff --git a/sql/hive/src/test/resources/golden/date_1-1-23edf29bf7376c70d5ecf12720f4b1eb b/sql/hive/src/test/resources/golden/date_1-0-23edf29bf7376c70d5ecf12720f4b1eb similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-1-23edf29bf7376c70d5ecf12720f4b1eb rename to sql/hive/src/test/resources/golden/date_1-0-23edf29bf7376c70d5ecf12720f4b1eb diff --git a/sql/hive/src/test/resources/golden/date_1-0-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/date_1-0-50131c0ba7b7a6b65c789a5a8497bada deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/date_1-0-50131c0ba7b7a6b65c789a5a8497bada +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/date_1-17-23edf29bf7376c70d5ecf12720f4b1eb b/sql/hive/src/test/resources/golden/date_1-1-4ebe3571c13a8b0c03096fbd972b7f1b similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-17-23edf29bf7376c70d5ecf12720f4b1eb rename to sql/hive/src/test/resources/golden/date_1-1-4ebe3571c13a8b0c03096fbd972b7f1b diff --git a/sql/hive/src/test/resources/golden/date_1-11-d964bec7e5632091ab5cb6f6786dbbf9 b/sql/hive/src/test/resources/golden/date_1-10-d964bec7e5632091ab5cb6f6786dbbf9 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-11-d964bec7e5632091ab5cb6f6786dbbf9 rename to sql/hive/src/test/resources/golden/date_1-10-d964bec7e5632091ab5cb6f6786dbbf9 diff --git a/sql/hive/src/test/resources/golden/date_1-12-480c5f024a28232b7857be327c992509 b/sql/hive/src/test/resources/golden/date_1-11-480c5f024a28232b7857be327c992509 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-12-480c5f024a28232b7857be327c992509 rename to sql/hive/src/test/resources/golden/date_1-11-480c5f024a28232b7857be327c992509 diff --git a/sql/hive/src/test/resources/golden/date_1-13-4c0ed7fcb75770d8790575b586bf14f4 b/sql/hive/src/test/resources/golden/date_1-12-4c0ed7fcb75770d8790575b586bf14f4 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-13-4c0ed7fcb75770d8790575b586bf14f4 rename to sql/hive/src/test/resources/golden/date_1-12-4c0ed7fcb75770d8790575b586bf14f4 diff --git a/sql/hive/src/test/resources/golden/date_1-14-44fc74c1993062c0a9522199ff27fea b/sql/hive/src/test/resources/golden/date_1-13-44fc74c1993062c0a9522199ff27fea similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-14-44fc74c1993062c0a9522199ff27fea rename to sql/hive/src/test/resources/golden/date_1-13-44fc74c1993062c0a9522199ff27fea diff --git a/sql/hive/src/test/resources/golden/date_1-15-4855a66124b16d1d0d003235995ac06b b/sql/hive/src/test/resources/golden/date_1-14-4855a66124b16d1d0d003235995ac06b similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-15-4855a66124b16d1d0d003235995ac06b rename to sql/hive/src/test/resources/golden/date_1-14-4855a66124b16d1d0d003235995ac06b diff --git a/sql/hive/src/test/resources/golden/date_1-16-8bc190dba0f641840b5e1e198a14c55b b/sql/hive/src/test/resources/golden/date_1-15-8bc190dba0f641840b5e1e198a14c55b similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-16-8bc190dba0f641840b5e1e198a14c55b rename to sql/hive/src/test/resources/golden/date_1-15-8bc190dba0f641840b5e1e198a14c55b diff --git a/sql/hive/src/test/resources/golden/date_1-2-4ebe3571c13a8b0c03096fbd972b7f1b b/sql/hive/src/test/resources/golden/date_1-16-23edf29bf7376c70d5ecf12720f4b1eb similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-2-4ebe3571c13a8b0c03096fbd972b7f1b rename to sql/hive/src/test/resources/golden/date_1-16-23edf29bf7376c70d5ecf12720f4b1eb diff --git a/sql/hive/src/test/resources/golden/date_1-3-26b5c291400dfde455b3c1b878b71d0 b/sql/hive/src/test/resources/golden/date_1-2-abdce0c0d14d3fc7441b7c134b02f99a similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-3-26b5c291400dfde455b3c1b878b71d0 rename to sql/hive/src/test/resources/golden/date_1-2-abdce0c0d14d3fc7441b7c134b02f99a diff --git a/sql/hive/src/test/resources/golden/date_1-10-df16364a220ff96a6ea1cd478cbc1d0b b/sql/hive/src/test/resources/golden/date_1-3-df16364a220ff96a6ea1cd478cbc1d0b similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-10-df16364a220ff96a6ea1cd478cbc1d0b rename to sql/hive/src/test/resources/golden/date_1-3-df16364a220ff96a6ea1cd478cbc1d0b diff --git a/sql/hive/src/test/resources/golden/date_1-5-d964bec7e5632091ab5cb6f6786dbbf9 b/sql/hive/src/test/resources/golden/date_1-4-d964bec7e5632091ab5cb6f6786dbbf9 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-5-d964bec7e5632091ab5cb6f6786dbbf9 rename to sql/hive/src/test/resources/golden/date_1-4-d964bec7e5632091ab5cb6f6786dbbf9 diff --git a/sql/hive/src/test/resources/golden/date_1-6-559d01fb0b42c42f0c4927fa0f9deac4 b/sql/hive/src/test/resources/golden/date_1-5-5e70fc74158fbfca38134174360de12d similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-6-559d01fb0b42c42f0c4927fa0f9deac4 rename to sql/hive/src/test/resources/golden/date_1-5-5e70fc74158fbfca38134174360de12d diff --git a/sql/hive/src/test/resources/golden/date_1-4-df16364a220ff96a6ea1cd478cbc1d0b b/sql/hive/src/test/resources/golden/date_1-6-df16364a220ff96a6ea1cd478cbc1d0b similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-4-df16364a220ff96a6ea1cd478cbc1d0b rename to sql/hive/src/test/resources/golden/date_1-6-df16364a220ff96a6ea1cd478cbc1d0b diff --git a/sql/hive/src/test/resources/golden/date_1-8-d964bec7e5632091ab5cb6f6786dbbf9 b/sql/hive/src/test/resources/golden/date_1-7-d964bec7e5632091ab5cb6f6786dbbf9 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-8-d964bec7e5632091ab5cb6f6786dbbf9 rename to sql/hive/src/test/resources/golden/date_1-7-d964bec7e5632091ab5cb6f6786dbbf9 diff --git a/sql/hive/src/test/resources/golden/date_1-9-8306558e0eabe936ac33dabaaa17fea4 b/sql/hive/src/test/resources/golden/date_1-8-1d5c58095cd52ea539d869f2ab1ab67d similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-9-8306558e0eabe936ac33dabaaa17fea4 rename to sql/hive/src/test/resources/golden/date_1-8-1d5c58095cd52ea539d869f2ab1ab67d diff --git a/sql/hive/src/test/resources/golden/date_1-7-df16364a220ff96a6ea1cd478cbc1d0b b/sql/hive/src/test/resources/golden/date_1-9-df16364a220ff96a6ea1cd478cbc1d0b similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-7-df16364a220ff96a6ea1cd478cbc1d0b rename to sql/hive/src/test/resources/golden/date_1-9-df16364a220ff96a6ea1cd478cbc1d0b 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..2d3ff680125ad 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive import java.util +import java.sql.Date import java.util.{Locale, TimeZone} import org.apache.hadoop.hive.ql.udf.UDAFPercentile @@ -75,7 +76,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(new Date(2014, 9, 23)) :: Literal(Decimal(BigDecimal(123.123))) :: Literal(new java.sql.Timestamp(123123)) :: Literal(Array[Byte](1,2,3)) :: @@ -142,6 +143,7 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { 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) } + case (r1: Date, r2: Date) => assert(r1.compareTo(r2) === 0) case (r1, r2) => assert(r1 === r2) } } 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 4f67d1def65fc..4c53b10ba96e9 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 @@ -253,30 +253,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("Cast Timestamp to Timestamp in UDF", """ - | SELECT DATEDIFF(CAST(value AS timestamp), CAST('2002-03-21 00:00:00' AS timestamp)) - | FROM src LIMIT 1 - """.stripMargin) - - createQueryTest("Date comparison test 1", - """ - | SELECT - | CAST(CAST('1970-01-01 22:00:00' AS timestamp) AS date) == - | CAST(CAST('1970-01-01 23:00:00' AS timestamp) AS date) - | FROM src LIMIT 1 - """.stripMargin) - - createQueryTest("Date comparison test 2", - "SELECT CAST(CAST(0 AS timestamp) AS date) > CAST(0 AS timestamp) FROM src LIMIT 1") - - createQueryTest("Date cast", - """ - | SELECT - | CAST(CAST(0 AS timestamp) AS date), - | CAST(CAST(CAST(0 AS timestamp) AS date) AS string), - | CAST(0 AS timestamp), - | CAST(CAST(0 AS timestamp) AS string), - | CAST(CAST(CAST('1970-01-01 23:00:00' AS timestamp) AS date) AS timestamp) - | FROM src LIMIT 1 + | SELECT DATEDIFF(CAST(value AS timestamp), CAST('2002-03-21 00:00:00' AS timestamp)) + | FROM src LIMIT 1 """.stripMargin) createQueryTest("Simple Average", 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..254919e8f6fdc 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 @@ -160,7 +160,7 @@ private[hive] object HiveShim { if (value == null) null else new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]]) def getDateWritable(value: Any): hiveIo.DateWritable = - if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[Int]) + if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[java.sql.Date]) def getTimestampWritable(value: Any): hiveIo.TimestampWritable = if (value == null) { 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..45ca59ae56a38 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 @@ -263,7 +263,7 @@ private[hive] object HiveShim { } def getDateWritable(value: Any): hiveIo.DateWritable = - if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[Int]) + if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[java.sql.Date]) def getTimestampWritable(value: Any): hiveIo.TimestampWritable = if (value == null) { From 554403fd913685da879cf6a280c58a9fad19448a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 2 Feb 2015 19:01:47 -0800 Subject: [PATCH 023/578] [SQL] Improve DataFrame API error reporting 1. Throw UnsupportedOperationException if a Column is not computable. 2. Perform eager analysis on DataFrame so we can catch errors when they happen (not when an action is run). Author: Reynold Xin Author: Davies Liu Closes #4296 from rxin/col-computability and squashes the following commits: 6527b86 [Reynold Xin] Merge pull request #8 from davies/col-computability fd92bc7 [Reynold Xin] Merge branch 'master' into col-computability f79034c [Davies Liu] fix python tests 5afe1ff [Reynold Xin] Fix scala test. 17f6bae [Reynold Xin] Various fixes. b932e86 [Reynold Xin] Added eager analysis for error reporting. e6f00b8 [Reynold Xin] [SQL][API] ComputableColumn vs IncomputableColumn --- python/pyspark/sql.py | 75 ++-- python/pyspark/tests.py | 6 +- .../apache/spark/sql/types/dataTypes.scala | 3 +- .../scala/org/apache/spark/sql/Column.scala | 241 ++++++++----- .../apache/spark/sql/ComputableColumn.scala | 33 ++ .../org/apache/spark/sql/DataFrame.scala | 292 ++++----------- .../org/apache/spark/sql/DataFrameImpl.scala | 331 ++++++++++++++++++ .../main/scala/org/apache/spark/sql/Dsl.scala | 21 +- .../apache/spark/sql/GroupedDataFrame.scala | 11 +- .../apache/spark/sql/IncomputableColumn.scala | 160 +++++++++ .../org/apache/spark/sql/SQLContext.scala | 18 +- .../apache/spark/sql/execution/commands.scala | 2 +- .../org/apache/spark/sql/sources/ddl.scala | 2 +- .../spark/sql/test/TestSQLContext.scala | 2 +- .../spark/sql/ColumnExpressionSuite.scala | 39 +++ .../org/apache/spark/sql/DataFrameSuite.scala | 13 + .../org/apache/spark/sql/SQLQuerySuite.scala | 6 +- .../sql/parquet/ParquetFilterSuite.scala | 6 +- .../apache/spark/sql/hive/HiveContext.scala | 3 +- .../spark/sql/hive/HiveStrategies.scala | 13 +- 20 files changed, 896 insertions(+), 381 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/ComputableColumn.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.scala diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 3f2d7ac82585f..32bff0c7e8c55 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -2124,6 +2124,10 @@ def head(self, n=None): return rs[0] if rs else None return self.take(n) + def first(self): + """ Return the first row. """ + return self.head() + def tail(self): raise NotImplemented @@ -2159,7 +2163,7 @@ def select(self, *cols): else: cols = [c._jc for c in cols] jcols = ListConverter().convert(cols, self._sc._gateway._gateway_client) - jdf = self._jdf.select(self._jdf.toColumnArray(jcols)) + jdf = self._jdf.select(self.sql_ctx._sc._jvm.Dsl.toColumns(jcols)) return DataFrame(jdf, self.sql_ctx) def filter(self, condition): @@ -2189,7 +2193,7 @@ def groupBy(self, *cols): else: cols = [c._jc for c in cols] jcols = ListConverter().convert(cols, self._sc._gateway._gateway_client) - jdf = self._jdf.groupBy(self._jdf.toColumnArray(jcols)) + jdf = self._jdf.groupBy(self.sql_ctx._sc._jvm.Dsl.toColumns(jcols)) return GroupedDataFrame(jdf, self.sql_ctx) def agg(self, *exprs): @@ -2278,14 +2282,17 @@ def agg(self, *exprs): :param exprs: list or aggregate columns or a map from column name to agregate methods. """ + 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 Columns" - jdf = self._jdf.agg(*exprs) + 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.Dsl.toColumns(jcols)) return DataFrame(jdf, self.sql_ctx) @dfapi @@ -2347,7 +2354,7 @@ def _create_column_from_literal(literal): def _create_column_from_name(name): sc = SparkContext._active_spark_context - return sc._jvm.Column(name) + return sc._jvm.IncomputableColumn(name) def _scalaMethod(name): @@ -2371,7 +2378,7 @@ def _(self): return _ -def _bin_op(name, pass_literal_through=False): +def _bin_op(name, pass_literal_through=True): """ Create a method for given binary operator Keyword arguments: @@ -2465,10 +2472,10 @@ def __init__(self, jc, jdf=None, sql_ctx=None): # __getattr__ = _bin_op("getField") # string methods - rlike = _bin_op("rlike", pass_literal_through=True) - like = _bin_op("like", pass_literal_through=True) - startswith = _bin_op("startsWith", pass_literal_through=True) - endswith = _bin_op("endsWith", pass_literal_through=True) + rlike = _bin_op("rlike") + like = _bin_op("like") + startswith = _bin_op("startsWith") + endswith = _bin_op("endsWith") upper = _unary_op("upper") lower = _unary_op("lower") @@ -2476,7 +2483,6 @@ def substr(self, startPos, pos): if type(startPos) != type(pos): raise TypeError("Can not mix the type") if isinstance(startPos, (int, long)): - jc = self._jc.substr(startPos, pos) elif isinstance(startPos, Column): jc = self._jc.substr(startPos._jc, pos._jc) @@ -2507,16 +2513,21 @@ def cast(self, dataType): return Column(self._jc.cast(jdt), self._jdf, self.sql_ctx) +def _to_java_column(col): + if isinstance(col, Column): + jcol = col._jc + else: + jcol = _create_column_from_name(col) + return jcol + + def _aggregate_func(name): """ Create a function for aggregator by name""" def _(col): sc = SparkContext._active_spark_context - if isinstance(col, Column): - jcol = col._jc - else: - jcol = _create_column_from_name(col) - jc = getattr(sc._jvm.org.apache.spark.sql.Dsl, name)(jcol) + jc = getattr(sc._jvm.Dsl, name)(_to_java_column(col)) return Column(jc) + return staticmethod(_) @@ -2524,13 +2535,31 @@ class Aggregator(object): """ A collections of builtin aggregators """ - max = _aggregate_func("max") - min = _aggregate_func("min") - avg = mean = _aggregate_func("mean") - sum = _aggregate_func("sum") - first = _aggregate_func("first") - last = _aggregate_func("last") - count = _aggregate_func("count") + AGGS = [ + 'lit', 'col', 'column', 'upper', 'lower', 'sqrt', 'abs', + 'min', 'max', 'first', 'last', 'count', 'avg', 'mean', 'sum', 'sumDistinct', + ] + for _name in AGGS: + locals()[_name] = _aggregate_func(_name) + del _name + + @staticmethod + def countDistinct(col, *cols): + 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.Dsl.toColumns(jcols)) + return Column(jc) + + @staticmethod + def approxCountDistinct(col, rsd=None): + 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) def _test(): diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index bec1961f26393..fef6c92875a1c 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -1029,9 +1029,11 @@ def test_aggregator(self): g = df.groupBy() self.assertEqual([99, 100], sorted(g.agg({'key': 'max', 'value': 'count'}).collect()[0])) self.assertEqual([Row(**{"AVG(key#0)": 49.5})], g.mean().collect()) - # TODO(davies): fix aggregators + from pyspark.sql import Aggregator as Agg - # self.assertEqual((0, '100'), tuple(g.agg(Agg.first(df.key), Agg.last(df.value)).first())) + self.assertEqual((0, u'99'), tuple(g.agg(Agg.first(df.key), Agg.last(df.value)).first())) + self.assertTrue(95 < g.agg(Agg.approxCountDistinct(df.key)).first()[0]) + self.assertEqual(100, g.agg(Agg.countDistinct(df.value)).first()[0]) def test_help_command(self): # Regression test for SPARK-5464 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 6ab99aa38877f..defdcb2b706f5 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 @@ -822,7 +822,8 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru * have a name matching the given name, `null` will be returned. */ def apply(name: String): StructField = { - nameToField.getOrElse(name, throw new IllegalArgumentException(s"Field $name does not exist.")) + nameToField.getOrElse(name, + throw new IllegalArgumentException(s"""Field "$name" does not exist.""")) } /** 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 174c403059510..6f48d7c3fe1b6 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,23 +17,26 @@ 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.catalyst.analysis.{UnresolvedStar, UnresolvedAttribute} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.{Project, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{Subquery, Project, LogicalPlan} import org.apache.spark.sql.types._ -object Column { - /** - * Creates a [[Column]] based on the given column name. Same as [[Dsl.col]]. - */ - def apply(colName: String): Column = new Column(colName) +private[sql] object Column { + + def apply(colName: String): Column = new IncomputableColumn(colName) + + def apply(expr: Expression): Column = new IncomputableColumn(expr) + + def apply(sqlContext: SQLContext, plan: LogicalPlan, expr: Expression): Column = { + new ComputableColumn(sqlContext, plan, expr) + } - /** For internal pattern matching. */ - private[sql] def unapply(col: Column): Option[Expression] = Some(col.expr) + def unapply(col: Column): Option[Expression] = Some(col.expr) } @@ -53,44 +56,42 @@ object Column { * */ // TODO: Improve documentation. -class Column( - sqlContext: Option[SQLContext], - plan: Option[LogicalPlan], - protected[sql] val expr: Expression) - extends DataFrame(sqlContext, plan) with ExpressionApi { +trait Column extends DataFrame with ExpressionApi { - /** Turns a Catalyst expression into a `Column`. */ - protected[sql] def this(expr: Expression) = this(None, None, expr) + protected[sql] def expr: Expression /** - * Creates a new `Column` expression based on a column or attribute name. - * The resolution of this is the same as SQL. For example: - * - * - "colName" becomes an expression selecting the column named "colName". - * - "*" becomes an expression selecting all columns. - * - "df.*" becomes an expression selecting all columns in data frame "df". + * Returns true iff the [[Column]] is computable. */ - 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) - }) + def isComputable: Boolean - override def isComputable: Boolean = sqlContext.isDefined && plan.isDefined + private def constructColumn(other: Column)(newExpr: 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 + } - /** - * An implicit conversion function internal to this class. This function creates a new Column - * based on an expression. If the expression itself is not named, it aliases the expression - * by calling it "col". - */ - private[this] implicit def toColumn(expr: Expression): Column = { - val projectedPlan = plan.map { p => - Project(Seq(expr match { + def computableCol(baseCol: ComputableColumn, expr: Expression) = { + val plan = Project(Seq(expr match { case named: NamedExpression => named case unnamed: Expression => Alias(unnamed, "col")() - }), p) + }), baseCol.plan) + Column(baseCol.sqlContext, plan, expr) + } + + (this, other) match { + case (left: ComputableColumn, right: ComputableColumn) => + if (stripProject(left.plan).sameResult(stripProject(right.plan))) { + computableCol(right, newExpr) + } else { + Column(newExpr) + } + case (left: ComputableColumn, _) => computableCol(left, newExpr) + case (_, right: ComputableColumn) => computableCol(right, newExpr) + case (_, _) => Column(newExpr) } - new Column(sqlContext, projectedPlan, expr) } /** @@ -100,7 +101,7 @@ class Column( * df.select( -df("amount") ) * }}} */ - override def unary_- : Column = UnaryMinus(expr) + override def unary_- : Column = constructColumn(null) { UnaryMinus(expr) } /** * Bitwise NOT. @@ -109,7 +110,7 @@ class Column( * df.select( ~df("flags") ) * }}} */ - override def unary_~ : Column = BitwiseNot(expr) + override def unary_~ : Column = constructColumn(null) { BitwiseNot(expr) } /** * Inversion of boolean expression, i.e. NOT. @@ -118,7 +119,7 @@ class Column( * df.select( !df("isActive") ) * }} */ - override def unary_! : Column = Not(expr) + override def unary_! : Column = constructColumn(null) { Not(expr) } /** @@ -129,7 +130,9 @@ class Column( * df.select( df("colA".equalTo(df("colB")) ) * }}} */ - override def === (other: Column): Column = EqualTo(expr, other.expr) + override def === (other: Column): Column = constructColumn(other) { + EqualTo(expr, other.expr) + } /** * Equality test with a literal value. @@ -169,7 +172,9 @@ class Column( * df.select( !(df("colA") === df("colB")) ) * }}} */ - override def !== (other: Column): Column = Not(EqualTo(expr, other.expr)) + override def !== (other: Column): Column = constructColumn(other) { + Not(EqualTo(expr, other.expr)) + } /** * Inequality test with a literal value. @@ -188,7 +193,9 @@ class Column( * people.select( people("age") > Literal(21) ) * }}} */ - override def > (other: Column): Column = GreaterThan(expr, other.expr) + override def > (other: Column): Column = constructColumn(other) { + GreaterThan(expr, other.expr) + } /** * Greater than a literal value. @@ -206,7 +213,9 @@ class Column( * people.select( people("age") < Literal(21) ) * }}} */ - override def < (other: Column): Column = LessThan(expr, other.expr) + override def < (other: Column): Column = constructColumn(other) { + LessThan(expr, other.expr) + } /** * Less than a literal value. @@ -224,7 +233,9 @@ class Column( * people.select( people("age") <= Literal(21) ) * }}} */ - override def <= (other: Column): Column = LessThanOrEqual(expr, other.expr) + override def <= (other: Column): Column = constructColumn(other) { + LessThanOrEqual(expr, other.expr) + } /** * Less than or equal to a literal value. @@ -242,7 +253,9 @@ class Column( * people.select( people("age") >= Literal(21) ) * }}} */ - override def >= (other: Column): Column = GreaterThanOrEqual(expr, other.expr) + override def >= (other: Column): Column = constructColumn(other) { + GreaterThanOrEqual(expr, other.expr) + } /** * Greater than or equal to a literal value. @@ -256,9 +269,11 @@ class Column( /** * Equality test with an expression that is safe for null values. */ - override def <=> (other: Column): Column = other match { - case null => EqualNullSafe(expr, lit(null).expr) - case _ => EqualNullSafe(expr, other.expr) + override def <=> (other: Column): Column = constructColumn(other) { + other match { + case null => EqualNullSafe(expr, lit(null).expr) + case _ => EqualNullSafe(expr, other.expr) + } } /** @@ -269,12 +284,12 @@ class Column( /** * True if the current expression is null. */ - override def isNull: Column = IsNull(expr) + override def isNull: Column = constructColumn(null) { IsNull(expr) } /** * True if the current expression is NOT null. */ - override def isNotNull: Column = IsNotNull(expr) + override def isNotNull: Column = constructColumn(null) { IsNotNull(expr) } /** * Boolean OR with an expression. @@ -283,7 +298,9 @@ class Column( * people.select( people("inSchool") || people("isEmployed") ) * }}} */ - override def || (other: Column): Column = Or(expr, other.expr) + override def || (other: Column): Column = constructColumn(other) { + Or(expr, other.expr) + } /** * Boolean OR with a literal value. @@ -301,7 +318,9 @@ class Column( * people.select( people("inSchool") && people("isEmployed") ) * }}} */ - override def && (other: Column): Column = And(expr, other.expr) + override def && (other: Column): Column = constructColumn(other) { + And(expr, other.expr) + } /** * Boolean AND with a literal value. @@ -315,7 +334,9 @@ class Column( /** * Bitwise AND with an expression. */ - override def & (other: Column): Column = BitwiseAnd(expr, other.expr) + override def & (other: Column): Column = constructColumn(other) { + BitwiseAnd(expr, other.expr) + } /** * Bitwise AND with a literal value. @@ -325,7 +346,9 @@ class Column( /** * Bitwise OR with an expression. */ - override def | (other: Column): Column = BitwiseOr(expr, other.expr) + override def | (other: Column): Column = constructColumn(other) { + BitwiseOr(expr, other.expr) + } /** * Bitwise OR with a literal value. @@ -335,7 +358,9 @@ class Column( /** * Bitwise XOR with an expression. */ - override def ^ (other: Column): Column = BitwiseXor(expr, other.expr) + override def ^ (other: Column): Column = constructColumn(other) { + BitwiseXor(expr, other.expr) + } /** * Bitwise XOR with a literal value. @@ -349,7 +374,9 @@ class Column( * people.select( people("height") + people("weight") ) * }}} */ - override def + (other: Column): Column = Add(expr, other.expr) + override def + (other: Column): Column = constructColumn(other) { + Add(expr, other.expr) + } /** * Sum of this expression and another expression. @@ -367,7 +394,9 @@ class Column( * people.select( people("height") - people("weight") ) * }}} */ - override def - (other: Column): Column = Subtract(expr, other.expr) + override def - (other: Column): Column = constructColumn(other) { + Subtract(expr, other.expr) + } /** * Subtraction. Subtract a literal value from this expression. @@ -385,7 +414,9 @@ class Column( * people.select( people("height") * people("weight") ) * }}} */ - override def * (other: Column): Column = Multiply(expr, other.expr) + override def * (other: Column): Column = constructColumn(other) { + Multiply(expr, other.expr) + } /** * Multiplication this expression and a literal value. @@ -403,7 +434,9 @@ class Column( * people.select( people("height") / people("weight") ) * }}} */ - override def / (other: Column): Column = Divide(expr, other.expr) + override def / (other: Column): Column = constructColumn(other) { + Divide(expr, other.expr) + } /** * Division this expression by a literal value. @@ -417,7 +450,9 @@ class Column( /** * Modulo (a.k.a. remainder) expression. */ - override def % (other: Column): Column = Remainder(expr, other.expr) + override def % (other: Column): Column = constructColumn(other) { + Remainder(expr, other.expr) + } /** * Modulo (a.k.a. remainder) expression. @@ -430,29 +465,40 @@ class Column( * by the evaluated values of the arguments. */ @scala.annotation.varargs - override def in(list: Column*): Column = In(expr, list.map(_.expr)) + override def in(list: Column*): Column = { + new IncomputableColumn(In(expr, list.map(_.expr))) + } - override def like(literal: String): Column = Like(expr, lit(literal).expr) + override def like(literal: String): Column = constructColumn(null) { + Like(expr, lit(literal).expr) + } - override def rlike(literal: String): Column = RLike(expr, lit(literal).expr) + override def rlike(literal: String): Column = constructColumn(null) { + RLike(expr, lit(literal).expr) + } /** * An expression that gets an item at position `ordinal` out of an array. */ - override def getItem(ordinal: Int): Column = GetItem(expr, Literal(ordinal)) + override def getItem(ordinal: Int): Column = constructColumn(null) { + GetItem(expr, Literal(ordinal)) + } /** * An expression that gets a field by name in a [[StructField]]. */ - override def getField(fieldName: String): Column = GetField(expr, fieldName) + override def getField(fieldName: String): Column = constructColumn(null) { + GetField(expr, fieldName) + } /** * An expression that returns a substring. * @param startPos expression for the starting position. * @param len expression for the length of the substring. */ - override def substr(startPos: Column, len: Column): Column = - Substring(expr, startPos.expr, len.expr) + override def substr(startPos: Column, len: Column): Column = { + new IncomputableColumn(Substring(expr, startPos.expr, len.expr)) + } /** * An expression that returns a substring. @@ -461,16 +507,21 @@ class Column( */ override def substr(startPos: Int, len: Int): Column = this.substr(lit(startPos), lit(len)) - override def contains(other: Column): Column = Contains(expr, other.expr) + override def contains(other: Column): Column = constructColumn(other) { + Contains(expr, other.expr) + } override def contains(literal: Any): Column = this.contains(lit(literal)) - - override def startsWith(other: Column): Column = StartsWith(expr, other.expr) + override def startsWith(other: Column): Column = constructColumn(other) { + StartsWith(expr, other.expr) + } override def startsWith(literal: String): Column = this.startsWith(lit(literal)) - override def endsWith(other: Column): Column = EndsWith(expr, other.expr) + override def endsWith(other: Column): Column = constructColumn(other) { + EndsWith(expr, other.expr) + } override def endsWith(literal: String): Column = this.endsWith(lit(literal)) @@ -481,7 +532,7 @@ class Column( * df.select($"colA".as("colB")) * }}} */ - override def as(alias: String): Column = Alias(expr, alias)() + override def as(alias: String): Column = constructColumn(null) { Alias(expr, alias)() } /** * Casts the column to a different data type. @@ -494,7 +545,7 @@ class Column( * df.select(df("colA").cast("int")) * }}} */ - override def cast(to: DataType): Column = Cast(expr, to) + override def cast(to: DataType): Column = constructColumn(null) { Cast(expr, to) } /** * Casts the column to a different data type, using the canonical string representation @@ -505,28 +556,30 @@ class Column( * df.select(df("colA").cast("int")) * }}} */ - override def cast(to: String): Column = 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"""") - }) - - override def desc: Column = SortOrder(expr, Descending) - - override def asc: Column = SortOrder(expr, Ascending) + override def cast(to: String): Column = constructColumn(null) { + 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"""") + }) + } + + override def desc: Column = constructColumn(null) { SortOrder(expr, Descending) } + + override def asc: Column = constructColumn(null) { SortOrder(expr, Ascending) } } -class ColumnName(name: String) extends Column(name) { +class ColumnName(name: String) extends IncomputableColumn(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 new file mode 100644 index 0000000000000..ac479b26a7c6a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/ComputableColumn.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 + +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 5d42d4428d09e..385e1ec74f5f7 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,26 +19,21 @@ package org.apache.spark.sql import java.util.{List => JList} -import scala.language.implicitConversions import scala.reflect.ClassTag -import scala.collection.JavaConversions._ -import com.fasterxml.jackson.core.JsonFactory - -import org.apache.spark.annotation.Experimental +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 -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.{LogicalRDD, EvaluatePython} -import org.apache.spark.sql.json.JsonRDD -import org.apache.spark.sql.types.{NumericType, StructType} -import org.apache.spark.util.Utils +import org.apache.spark.sql.types.StructType + + +private[sql] object DataFrame { + def apply(sqlContext: SQLContext, logicalPlan: LogicalPlan): DataFrame = { + new DataFrameImpl(sqlContext, logicalPlan) + } +} /** @@ -78,50 +73,14 @@ import org.apache.spark.util.Utils * }}} */ // TODO: Improve documentation. -class DataFrame protected[sql]( - val sqlContext: SQLContext, - private val baseLogicalPlan: LogicalPlan, - operatorsEnabled: Boolean) - extends DataFrameSpecificApi with RDDApi[Row] { - - protected[sql] def this(sqlContext: Option[SQLContext], plan: Option[LogicalPlan]) = - this(sqlContext.orNull, plan.orNull, sqlContext.isDefined && plan.isDefined) - - protected[sql] def this(sqlContext: SQLContext, plan: LogicalPlan) = this(sqlContext, plan, true) - - @transient protected[sql] lazy val queryExecution = sqlContext.executePlan(baseLogicalPlan) - - @transient protected[sql] val logicalPlan: LogicalPlan = baseLogicalPlan 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 => - LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext) - case _ => - baseLogicalPlan - } +trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { - /** - * An implicit conversion function internal to this class for us to avoid doing - * "new DataFrame(...)" everywhere. - */ - private implicit def logicalPlanToDataFrame(logicalPlan: LogicalPlan): DataFrame = { - new DataFrame(sqlContext, logicalPlan, true) - } + val sqlContext: SQLContext - /** Returns the list of numeric columns, useful for doing aggregation. */ - protected[sql] def numericColumns: Seq[Expression] = { - schema.fields.filter(_.dataType.isInstanceOf[NumericType]).map { n => - queryExecution.analyzed.resolve(n.name, sqlContext.analyzer.resolver).get - } - } + @DeveloperApi + def queryExecution: SQLContext#QueryExecution - /** Resolves a column name into a Catalyst [[NamedExpression]]. */ - protected[sql] def resolve(colName: String): NamedExpression = { - queryExecution.analyzed.resolve(colName, sqlContext.analyzer.resolver).getOrElse { - throw new RuntimeException( - s"""Cannot resolve column name "$colName" among (${schema.fieldNames.mkString(", ")})""") - } - } + protected[sql] def logicalPlan: LogicalPlan /** Left here for compatibility reasons. */ @deprecated("1.3.0", "use toDataFrame") @@ -142,32 +101,19 @@ class DataFrame protected[sql]( * }}} */ @scala.annotation.varargs - def toDataFrame(colName: String, colNames: String*): DataFrame = { - val newNames = colName +: colNames - require(schema.size == newNames.size, - "The number of columns doesn't match.\n" + - "Old column names: " + schema.fields.map(_.name).mkString(", ") + "\n" + - "New column names: " + newNames.mkString(", ")) - - val newCols = schema.fieldNames.zip(newNames).map { case (oldName, newName) => - apply(oldName).as(newName) - } - select(newCols :_*) - } + def toDataFrame(colName: String, colNames: String*): DataFrame /** Returns the schema of this [[DataFrame]]. */ - override def schema: StructType = queryExecution.analyzed.schema + override def schema: StructType /** Returns all column names and their data types as an array. */ - override def dtypes: Array[(String, String)] = schema.fields.map { field => - (field.name, field.dataType.toString) - } + override def dtypes: Array[(String, String)] /** Returns all column names as an array. */ override def columns: Array[String] = schema.fields.map(_.name) /** Prints the schema to the console in a nice tree format. */ - override def printSchema(): Unit = println(schema.treeString) + override def printSchema(): Unit /** * Cartesian join with another [[DataFrame]]. @@ -176,9 +122,7 @@ class DataFrame protected[sql]( * * @param right Right side of the join operation. */ - override def join(right: DataFrame): DataFrame = { - Join(logicalPlan, right.logicalPlan, joinType = Inner, None) - } + override def join(right: DataFrame): DataFrame /** * Inner join with another [[DataFrame]], using the given join expression. @@ -189,9 +133,7 @@ class DataFrame protected[sql]( * df1.join(df2).where($"df1Key" === $"df2Key") * }}} */ - override def join(right: DataFrame, joinExprs: Column): DataFrame = { - Join(logicalPlan, right.logicalPlan, Inner, Some(joinExprs.expr)) - } + override def join(right: DataFrame, joinExprs: Column): DataFrame /** * Join with another [[DataFrame]], usin g the given join expression. The following performs @@ -205,9 +147,7 @@ class DataFrame protected[sql]( * @param joinExprs Join expression. * @param joinType One of: `inner`, `outer`, `left_outer`, `right_outer`, `semijoin`. */ - override def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame = { - Join(logicalPlan, right.logicalPlan, JoinType(joinType), Some(joinExprs.expr)) - } + override def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame /** * Returns a new [[DataFrame]] sorted by the specified column, all in ascending order. @@ -219,9 +159,7 @@ class DataFrame protected[sql]( * }}} */ @scala.annotation.varargs - override def sort(sortCol: String, sortCols: String*): DataFrame = { - orderBy(apply(sortCol), sortCols.map(apply) :_*) - } + override def sort(sortCol: String, sortCols: String*): DataFrame /** * Returns a new [[DataFrame]] sorted by the given expressions. For example: @@ -230,46 +168,26 @@ class DataFrame protected[sql]( * }}} */ @scala.annotation.varargs - override def sort(sortExpr: Column, sortExprs: Column*): DataFrame = { - val sortOrder: Seq[SortOrder] = (sortExpr +: sortExprs).map { col => - col.expr match { - case expr: SortOrder => - expr - case expr: Expression => - SortOrder(expr, Ascending) - } - } - Sort(sortOrder, global = true, logicalPlan) - } + override def sort(sortExpr: Column, sortExprs: Column*): DataFrame /** * Returns a new [[DataFrame]] sorted by the given expressions. * This is an alias of the `sort` function. */ @scala.annotation.varargs - override def orderBy(sortCol: String, sortCols: String*): DataFrame = { - sort(sortCol, sortCols :_*) - } + override def orderBy(sortCol: String, sortCols: String*): DataFrame /** * Returns a new [[DataFrame]] sorted by the given expressions. * This is an alias of the `sort` function. */ @scala.annotation.varargs - override def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame = { - sort(sortExpr, sortExprs :_*) - } + override def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame /** * Selects column based on the column name and return it as a [[Column]]. */ - override def apply(colName: String): Column = colName match { - case "*" => - new Column(ResolvedStar(schema.fieldNames.map(resolve))) - case _ => - val expr = resolve(colName) - new Column(Some(sqlContext), Some(Project(Seq(expr), logicalPlan)), expr) - } + override def apply(colName: String): Column /** * Selects a set of expressions, wrapped in a Product. @@ -279,18 +197,12 @@ class DataFrame protected[sql]( * df.select($"colA", $"colB" + 1) * }}} */ - override def apply(projection: Product): DataFrame = { - require(projection.productArity >= 1) - select(projection.productIterator.map { - case c: Column => c - case o: Any => new Column(Some(sqlContext), None, Literal(o)) - }.toSeq :_*) - } + override def apply(projection: Product): DataFrame /** * Returns a new [[DataFrame]] with an alias set. */ - override def as(name: String): DataFrame = Subquery(name, logicalPlan) + override def as(name: String): DataFrame /** * Selects a set of expressions. @@ -299,15 +211,7 @@ class DataFrame protected[sql]( * }}} */ @scala.annotation.varargs - override def select(cols: Column*): DataFrame = { - val exprs = cols.zipWithIndex.map { - case (Column(expr: NamedExpression), _) => - expr - case (Column(expr: Expression), _) => - Alias(expr, expr.toString)() - } - Project(exprs.toSeq, logicalPlan) - } + override def select(cols: Column*): DataFrame /** * Selects a set of columns. This is a variant of `select` that can only select @@ -320,9 +224,7 @@ class DataFrame protected[sql]( * }}} */ @scala.annotation.varargs - override def select(col: String, cols: String*): DataFrame = { - select((col +: cols).map(new Column(_)) :_*) - } + override def select(col: String, cols: String*): DataFrame /** * Filters rows using the given condition. @@ -333,9 +235,7 @@ class DataFrame protected[sql]( * peopleDf($"age" > 15) * }}} */ - override def filter(condition: Column): DataFrame = { - Filter(condition.expr, logicalPlan) - } + override def filter(condition: Column): DataFrame /** * Filters rows using the given condition. This is an alias for `filter`. @@ -346,7 +246,7 @@ class DataFrame protected[sql]( * peopleDf($"age" > 15) * }}} */ - override def where(condition: Column): DataFrame = filter(condition) + override def where(condition: Column): DataFrame /** * Filters rows using the given condition. This is a shorthand meant for Scala. @@ -357,7 +257,7 @@ class DataFrame protected[sql]( * peopleDf($"age" > 15) * }}} */ - override def apply(condition: Column): DataFrame = filter(condition) + override def apply(condition: Column): DataFrame /** * Groups the [[DataFrame]] using the specified columns, so we can run aggregation on them. @@ -375,9 +275,7 @@ class DataFrame protected[sql]( * }}} */ @scala.annotation.varargs - override def groupBy(cols: Column*): GroupedDataFrame = { - new GroupedDataFrame(this, cols.map(_.expr)) - } + override def groupBy(cols: Column*): GroupedDataFrame /** * Groups the [[DataFrame]] using the specified columns, so we can run aggregation on them. @@ -398,10 +296,7 @@ class DataFrame protected[sql]( * }}} */ @scala.annotation.varargs - override def groupBy(col1: String, cols: String*): GroupedDataFrame = { - val colNames: Seq[String] = col1 +: cols - new GroupedDataFrame(this, colNames.map(colName => resolve(colName))) - } + override def groupBy(col1: String, cols: String*): GroupedDataFrame /** * Aggregates on the entire [[DataFrame]] without groups. @@ -411,7 +306,7 @@ class DataFrame protected[sql]( * df.groupBy().agg(Map("age" -> "max", "salary" -> "avg")) * }} */ - override def agg(exprs: Map[String, String]): DataFrame = groupBy().agg(exprs) + override def agg(exprs: Map[String, String]): DataFrame /** * Aggregates on the entire [[DataFrame]] without groups. @@ -421,7 +316,7 @@ class DataFrame protected[sql]( * df.groupBy().agg(Map("age" -> "max", "salary" -> "avg")) * }} */ - override def agg(exprs: java.util.Map[String, String]): DataFrame = agg(exprs.toMap) + override def agg(exprs: java.util.Map[String, String]): DataFrame /** * Aggregates on the entire [[DataFrame]] without groups. @@ -432,31 +327,31 @@ class DataFrame protected[sql]( * }} */ @scala.annotation.varargs - override def agg(expr: Column, exprs: Column*): DataFrame = groupBy().agg(expr, exprs :_*) + override def agg(expr: Column, exprs: Column*): DataFrame /** * 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]]. */ - override def limit(n: Int): DataFrame = Limit(Literal(n), logicalPlan) + override 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. */ - override def unionAll(other: DataFrame): DataFrame = Union(logicalPlan, other.logicalPlan) + override 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. */ - override def intersect(other: DataFrame): DataFrame = Intersect(logicalPlan, other.logicalPlan) + override 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. */ - override def except(other: DataFrame): DataFrame = Except(logicalPlan, other.logicalPlan) + override def except(other: DataFrame): DataFrame /** * Returns a new [[DataFrame]] by sampling a fraction of rows. @@ -465,9 +360,7 @@ class DataFrame protected[sql]( * @param fraction Fraction of rows to generate. * @param seed Seed for sampling. */ - override def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame = { - Sample(fraction, withReplacement, seed, logicalPlan) - } + override def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame /** * Returns a new [[DataFrame]] by sampling a fraction of rows, using a random seed. @@ -475,105 +368,85 @@ class DataFrame protected[sql]( * @param withReplacement Sample with replacement or not. * @param fraction Fraction of rows to generate. */ - override def sample(withReplacement: Boolean, fraction: Double): DataFrame = { - sample(withReplacement, fraction, Utils.random.nextLong) - } + override def sample(withReplacement: Boolean, fraction: Double): DataFrame ///////////////////////////////////////////////////////////////////////////// /** * Returns a new [[DataFrame]] by adding a column. */ - override def addColumn(colName: String, col: Column): DataFrame = { - select(Column("*"), col.as(colName)) - } + override def addColumn(colName: String, col: Column): DataFrame /** * Returns the first `n` rows. */ - override def head(n: Int): Array[Row] = limit(n).collect() + override def head(n: Int): Array[Row] /** * Returns the first row. */ - override def head(): Row = head(1).head + override def head(): Row /** * Returns the first row. Alias for head(). */ - override def first(): Row = head() + override def first(): Row /** * Returns a new RDD by applying a function to all rows of this DataFrame. */ - override def map[R: ClassTag](f: Row => R): RDD[R] = { - rdd.map(f) - } + 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. */ - override def flatMap[R: ClassTag](f: Row => TraversableOnce[R]): RDD[R] = rdd.flatMap(f) + 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. */ - override def mapPartitions[R: ClassTag](f: Iterator[Row] => Iterator[R]): RDD[R] = { - rdd.mapPartitions(f) - } - + override def mapPartitions[R: ClassTag](f: Iterator[Row] => Iterator[R]): RDD[R] /** * Applies a function `f` to all rows. */ - override def foreach(f: Row => Unit): Unit = rdd.foreach(f) + override def foreach(f: Row => Unit): Unit /** * Applies a function f to each partition of this [[DataFrame]]. */ - override def foreachPartition(f: Iterator[Row] => Unit): Unit = rdd.foreachPartition(f) + override def foreachPartition(f: Iterator[Row] => Unit): Unit /** * Returns the first `n` rows in the [[DataFrame]]. */ - override def take(n: Int): Array[Row] = head(n) + override def take(n: Int): Array[Row] /** * Returns an array that contains all of [[Row]]s in this [[DataFrame]]. */ - override def collect(): Array[Row] = queryExecution.executedPlan.executeCollect() + override def collect(): Array[Row] /** * Returns a Java list that contains all of [[Row]]s in this [[DataFrame]]. */ - override def collectAsList(): java.util.List[Row] = java.util.Arrays.asList(rdd.collect() :_*) + override def collectAsList(): java.util.List[Row] /** * Returns the number of rows in the [[DataFrame]]. */ - override def count(): Long = groupBy().count().rdd.collect().head.getLong(0) + override def count(): Long /** * Returns a new [[DataFrame]] that has exactly `numPartitions` partitions. */ - override def repartition(numPartitions: Int): DataFrame = { - sqlContext.applySchema(rdd.repartition(numPartitions), schema) - } + override def repartition(numPartitions: Int): DataFrame - override def persist(): this.type = { - sqlContext.cacheManager.cacheQuery(this) - this - } + override def persist(): this.type - override def persist(newLevel: StorageLevel): this.type = { - sqlContext.cacheManager.cacheQuery(this, None, newLevel) - this - } + override def persist(newLevel: StorageLevel): this.type - override def unpersist(blocking: Boolean): this.type = { - sqlContext.cacheManager.tryUncacheQuery(this, blocking) - this - } + override def unpersist(blocking: Boolean): this.type ///////////////////////////////////////////////////////////////////////////// // I/O @@ -582,10 +455,7 @@ class DataFrame protected[sql]( /** * Returns the content of the [[DataFrame]] as an [[RDD]] of [[Row]]s. */ - override def rdd: RDD[Row] = { - val schema = this.schema - queryExecution.executedPlan.execute().map(ScalaReflection.convertRowToScala(_, schema)) - } + override def rdd: RDD[Row] /** * Registers this RDD as a temporary table using the given name. The lifetime of this temporary @@ -593,18 +463,14 @@ class DataFrame protected[sql]( * * @group schema */ - override def registerTempTable(tableName: String): Unit = { - sqlContext.registerRDDAsTable(this, tableName) - } + override def registerTempTable(tableName: String): Unit /** * 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]]. */ - override def saveAsParquetFile(path: String): Unit = { - sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd - } + override def saveAsParquetFile(path: String): Unit /** * :: Experimental :: @@ -617,48 +483,26 @@ class DataFrame protected[sql]( * be the target of an `insertInto`. */ @Experimental - override def saveAsTable(tableName: String): Unit = { - sqlContext.executePlan( - CreateTableAsSelect(None, tableName, logicalPlan, allowExisting = false)).toRdd - } + override def saveAsTable(tableName: String): Unit /** * :: Experimental :: * Adds the rows from this RDD to the specified table, optionally overwriting the existing data. */ @Experimental - override def insertInto(tableName: String, overwrite: Boolean): Unit = { - sqlContext.executePlan(InsertIntoTable(UnresolvedRelation(Seq(tableName)), - Map.empty, logicalPlan, overwrite)).toRdd - } + override def insertInto(tableName: String, overwrite: Boolean): Unit /** * Returns the content of the [[DataFrame]] as a RDD of JSON strings. */ - override def toJSON: RDD[String] = { - val rowSchema = this.schema - this.mapPartitions { iter => - val jsonFactory = new JsonFactory() - iter.map(JsonRDD.rowToJSON(rowSchema, jsonFactory)) - } - } + override def toJSON: RDD[String] //////////////////////////////////////////////////////////////////////////// // for Python API //////////////////////////////////////////////////////////////////////////// - /** - * A helpful function for Py4j, convert a list of Column to an array - */ - protected[sql] def toColumnArray(cols: JList[Column]): Array[Column] = { - cols.toList.toArray - } /** * Converts a JavaRDD to a PythonRDD. */ - protected[sql] def javaToPython: JavaRDD[Array[Byte]] = { - val fieldTypes = schema.fields.map(_.dataType) - val jrdd = rdd.map(EvaluatePython.rowToArray(_, fieldTypes)).toJavaRDD() - SerDeUtil.javaToPython(jrdd) - } + protected[sql] def javaToPython: JavaRDD[Array[Byte]] } 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 new file mode 100644 index 0000000000000..f8fcc25569482 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameImpl.scala @@ -0,0 +1,331 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF 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.util.{List => JList} + +import scala.language.implicitConversions +import scala.reflect.ClassTag +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.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.{LogicalRDD, EvaluatePython} +import org.apache.spark.sql.json.JsonRDD +import org.apache.spark.sql.types.{NumericType, StructType} +import org.apache.spark.util.Utils + + +/** + * See [[DataFrame]] for documentation. + */ +private[sql] class DataFrameImpl protected[sql]( + override val sqlContext: SQLContext, + val queryExecution: SQLContext#QueryExecution) + extends DataFrame { + + def this(sqlContext: SQLContext, logicalPlan: LogicalPlan) = { + this(sqlContext, { + val qe = sqlContext.executePlan(logicalPlan) + 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[_] |_: 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 RuntimeException( + 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 toDataFrame(colName: String, colNames: String*): DataFrame = { + val newNames = colName +: colNames + require(schema.size == newNames.size, + "The number of columns doesn't match.\n" + + "Old column names: " + schema.fields.map(_.name).mkString(", ") + "\n" + + "New column names: " + newNames.mkString(", ")) + + val newCols = schema.fieldNames.zip(newNames).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 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 = { + orderBy(apply(sortCol), sortCols.map(apply) :_*) + } + + override def sort(sortExpr: Column, sortExprs: Column*): DataFrame = { + val sortOrder: Seq[SortOrder] = (sortExpr +: 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(sortExpr: Column, sortExprs: Column*): DataFrame = { + sort(sortExpr, sortExprs :_*) + } + + override def apply(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 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(name: String): DataFrame = Subquery(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)() + } + Project(exprs.toSeq, logicalPlan) + } + + override def select(col: String, cols: String*): DataFrame = { + select((col +: cols).map(Column(_)) :_*) + } + + override def filter(condition: Column): DataFrame = { + Filter(condition.expr, logicalPlan) + } + + override def where(condition: Column): DataFrame = { + filter(condition) + } + + override def apply(condition: Column): DataFrame = { + filter(condition) + } + + override def groupBy(cols: Column*): GroupedDataFrame = { + new GroupedDataFrame(this, cols.map(_.expr)) + } + + override def groupBy(col1: String, cols: String*): GroupedDataFrame = { + val colNames: Seq[String] = col1 +: cols + new GroupedDataFrame(this, colNames.map(colName => resolve(colName))) + } + + override def agg(exprs: Map[String, String]): DataFrame = { + groupBy().agg(exprs) + } + + override def agg(exprs: java.util.Map[String, String]): DataFrame = { + agg(exprs.toMap) + } + + override def agg(expr: Column, exprs: Column*): DataFrame = { + groupBy().agg(expr, exprs :_*) + } + + 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 sample(withReplacement: Boolean, fraction: Double): DataFrame = { + sample(withReplacement, fraction, Utils.random.nextLong) + } + + ///////////////////////////////////////////////////////////////////////////// + + override def addColumn(colName: String, col: Column): DataFrame = { + select(Column("*"), col.as(colName)) + } + + 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.applySchema(rdd.repartition(numPartitions), schema) + } + + 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] = { + val schema = this.schema + queryExecution.executedPlan.execute().map(ScalaReflection.convertRowToScala(_, schema)) + } + + override def registerTempTable(tableName: String): Unit = { + sqlContext.registerRDDAsTable(this, tableName) + } + + override def saveAsParquetFile(path: String): Unit = { + sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd + } + + override def saveAsTable(tableName: String): Unit = { + sqlContext.executePlan( + CreateTableAsSelect(None, tableName, logicalPlan, allowExisting = false)).toRdd + } + + 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 jsonFactory = new JsonFactory() + iter.map(JsonRDD.rowToJSON(rowSchema, jsonFactory)) + } + } + + //////////////////////////////////////////////////////////////////////////// + // 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/Dsl.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dsl.scala index 3499956023d11..b4279a32ffa21 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 @@ -17,8 +17,11 @@ package org.apache.spark.sql +import java.util.{List => JList} + import scala.language.implicitConversions import scala.reflect.runtime.universe.{TypeTag, typeTag} +import scala.collection.JavaConversions._ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions._ @@ -55,17 +58,17 @@ object Dsl { } } - private[this] implicit def toColumn(expr: Expression): Column = new Column(expr) + private[this] implicit def toColumn(expr: Expression): Column = Column(expr) /** * Returns a [[Column]] based on the given column name. */ - def col(colName: String): Column = new Column(colName) + def col(colName: String): Column = Column(colName) /** * Returns a [[Column]] based on the given column name. Alias of [[col]]. */ - def column(colName: String): Column = new Column(colName) + def column(colName: String): Column = Column(colName) /** * Creates a [[Column]] of literal value. @@ -94,7 +97,7 @@ object Dsl { case _ => throw new RuntimeException("Unsupported literal type " + literal.getClass + " " + literal) } - new Column(literalExpr) + Column(literalExpr) } def sum(e: Column): Column = Sum(e.expr) @@ -105,8 +108,7 @@ object Dsl { def countDistinct(expr: Column, exprs: Column*): Column = CountDistinct((expr +: exprs).map(_.expr)) - def approxCountDistinct(e: Column): Column = - ApproxCountDistinct(e.expr) + def approxCountDistinct(e: Column): Column = ApproxCountDistinct(e.expr) def approxCountDistinct(e: Column, rsd: Double): Column = ApproxCountDistinct(e.expr, rsd) @@ -121,6 +123,13 @@ object Dsl { def sqrt(e: Column): Column = Sqrt(e.expr) def abs(e: Column): Column = Abs(e.expr) + /** + * This is a private API for Python + * TODO: move this to a private package + */ + def toColumns(cols: JList[Column]): Seq[Column] = { + cols.toList.toSeq + } // scalastyle:off diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala index 1c948cbbfe58f..d3acd41bbf3eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql +import java.util.{List => JList} + import scala.language.implicitConversions import scala.collection.JavaConversions._ @@ -28,7 +30,7 @@ import org.apache.spark.sql.catalyst.plans.logical.Aggregate /** * A set of methods for aggregations on a [[DataFrame]], created by [[DataFrame.groupBy]]. */ -class GroupedDataFrame protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) +class GroupedDataFrame protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expression]) extends GroupedDataFrameApi { private[this] implicit def toDataFrame(aggExprs: Seq[NamedExpression]): DataFrame = { @@ -36,8 +38,8 @@ class GroupedDataFrame protected[sql](df: DataFrame, groupingExprs: Seq[Expressi case expr: NamedExpression => expr case expr: Expression => Alias(expr, expr.toString)() } - new DataFrame(df.sqlContext, - Aggregate(groupingExprs, namedGroupingExprs ++ aggExprs, df.logicalPlan)) + DataFrame( + df.sqlContext, Aggregate(groupingExprs, namedGroupingExprs ++ aggExprs, df.logicalPlan)) } private[this] def aggregateNumericColumns(f: Expression => Expression): Seq[NamedExpression] = { @@ -112,8 +114,7 @@ class GroupedDataFrame protected[sql](df: DataFrame, groupingExprs: Seq[Expressi case expr: NamedExpression => expr case expr: Expression => Alias(expr, expr.toString)() } - - new DataFrame(df.sqlContext, Aggregate(groupingExprs, aggExprs, df.logicalPlan)) + DataFrame(df.sqlContext, Aggregate(groupingExprs, aggExprs, df.logicalPlan)) } /** 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 new file mode 100644 index 0000000000000..2f8c695d5654b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/IncomputableColumn.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.sql + +import scala.reflect.ClassTag + +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 isComputable: Boolean = false + + override val sqlContext: SQLContext = null + + override def queryExecution = err() + + protected[sql] override def logicalPlan: LogicalPlan = err() + + override def toDataFrame(colName: String, 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 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(sortExpr: Column, sortExprs: Column*): DataFrame = err() + + override def orderBy(sortCol: String, sortCols: String*): DataFrame = err() + + override def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame = err() + + override def apply(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() + + override def filter(condition: Column): DataFrame = err() + + override def where(condition: Column): DataFrame = err() + + override def apply(condition: Column): DataFrame = err() + + override def groupBy(cols: Column*): GroupedDataFrame = err() + + override def groupBy(col1: String, cols: String*): GroupedDataFrame = err() + + override def agg(exprs: Map[String, String]): DataFrame = err() + + override def agg(exprs: java.util.Map[String, String]): DataFrame = err() + + override def agg(expr: Column, exprs: Column*): DataFrame = 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 sample(withReplacement: Boolean, fraction: Double): DataFrame = err() + + ///////////////////////////////////////////////////////////////////////////// + + override def addColumn(colName: String, col: Column): 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 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): Unit = err() + + override def insertInto(tableName: 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 84933dd944837..d0bbb5f7a34f1 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 @@ -171,14 +171,14 @@ class SQLContext(@transient val sparkContext: SparkContext) val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] val attributeSeq = schema.toAttributes val rowRDD = RDDConversions.productToRowRdd(rdd, schema) - new DataFrame(this, LogicalRDD(attributeSeq, rowRDD)(self)) + DataFrame(this, LogicalRDD(attributeSeq, rowRDD)(self)) } /** * Convert a [[BaseRelation]] created for external data sources into a [[DataFrame]]. */ def baseRelationToDataFrame(baseRelation: BaseRelation): DataFrame = { - new DataFrame(this, LogicalRelation(baseRelation)) + DataFrame(this, LogicalRelation(baseRelation)) } /** @@ -216,7 +216,7 @@ 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 logicalPlan = LogicalRDD(schema.toAttributes, rowRDD)(self) - new DataFrame(this, logicalPlan) + DataFrame(this, logicalPlan) } /** @@ -243,7 +243,7 @@ class SQLContext(@transient val sparkContext: SparkContext) ) : Row } } - new DataFrame(this, LogicalRDD(attributeSeq, rowRdd)(this)) + DataFrame(this, LogicalRDD(attributeSeq, rowRdd)(this)) } /** @@ -262,7 +262,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ def parquetFile(path: String): DataFrame = - new DataFrame(this, parquet.ParquetRelation(path, Some(sparkContext.hadoopConfiguration), this)) + DataFrame(this, parquet.ParquetRelation(path, Some(sparkContext.hadoopConfiguration), this)) /** * Loads a JSON file (one object per line), returning the result as a [[DataFrame]]. @@ -365,7 +365,7 @@ class SQLContext(@transient val sparkContext: SparkContext) */ def sql(sqlText: String): DataFrame = { if (conf.dialect == "sql") { - new DataFrame(this, parseSql(sqlText)) + DataFrame(this, parseSql(sqlText)) } else { sys.error(s"Unsupported SQL dialect: ${conf.dialect}") } @@ -373,7 +373,7 @@ class SQLContext(@transient val sparkContext: SparkContext) /** Returns the specified table as a [[DataFrame]]. */ def table(tableName: String): DataFrame = - new DataFrame(this, catalog.lookupRelation(Seq(tableName))) + DataFrame(this, catalog.lookupRelation(Seq(tableName))) protected[sql] class SparkPlanner extends SparkStrategies { val sparkContext: SparkContext = self.sparkContext @@ -462,7 +462,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * access to the intermediate phases of query execution for developers. */ @DeveloperApi - protected class QueryExecution(val logical: LogicalPlan) { + protected[sql] class QueryExecution(val logical: LogicalPlan) { lazy val analyzed: LogicalPlan = ExtractPythonUdfs(analyzer(logical)) lazy val withCachedData: LogicalPlan = cacheManager.useCachedData(analyzed) @@ -556,7 +556,7 @@ class SQLContext(@transient val sparkContext: SparkContext) iter.map { m => new GenericRow(m): Row} } - new DataFrame(this, LogicalRDD(schema.toAttributes, rowRdd)(self)) + DataFrame(this, LogicalRDD(schema.toAttributes, rowRdd)(self)) } /** 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 6fba76c52171b..e1c9a2be7d20d 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 @@ -138,7 +138,7 @@ case class CacheTableCommand( override def run(sqlContext: SQLContext) = { plan.foreach { logicalPlan => - sqlContext.registerRDDAsTable(new DataFrame(sqlContext, logicalPlan), tableName) + sqlContext.registerRDDAsTable(DataFrame(sqlContext, logicalPlan), tableName) } sqlContext.cacheTable(tableName) 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 b7c721f8c0691..b1bbe0f89af73 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 @@ -231,7 +231,7 @@ private [sql] case class CreateTempTableUsing( def run(sqlContext: SQLContext) = { val resolved = ResolvedDataSource(sqlContext, userSpecifiedSchema, provider, options) sqlContext.registerRDDAsTable( - new DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName) + DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName) Seq.empty } } 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 906455dd40c0d..4e1ec38bd0158 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 @@ -41,7 +41,7 @@ object TestSQLContext * construct [[DataFrame]] directly out of local data without relying on implicits. */ protected[sql] implicit def logicalPlanToSparkQuery(plan: LogicalPlan): DataFrame = { - new DataFrame(this, plan) + DataFrame(this, plan) } } 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 2d464c2b53d79..fa4cdecbcb340 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 @@ -27,6 +27,45 @@ 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")) + + shouldBeComputable(testData2.select(($"a" + 1).as("c"))("c") + testData2("b")) + shouldBeComputable( + testData2.select(($"a" + 1).as("c"))("c") + testData2.select(($"b" / 2).as("d"))("d")) + shouldBeComputable( + 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("star") { checkAnswer(testData.select($"*"), testData.collect().toSeq) } 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 df343adc793bd..f6b65a81ce05e 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 @@ -28,6 +28,19 @@ import scala.language.postfixOps class DataFrameSuite extends QueryTest { import org.apache.spark.sql.TestData._ + test("analysis error should be eagerly reported") { + intercept[Exception] { testData.select('nonExistentName) } + intercept[Exception] { + testData.groupBy('key).agg(Map("nonExistentName" -> "sum")) + } + intercept[Exception] { + testData.groupBy("nonExistentName").agg(Map("key" -> "sum")) + } + intercept[Exception] { + testData.groupBy($"abcd").agg(Map("key" -> "sum")) + } + } + test("table scan") { checkAnswer( testData, 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 d82c34316cefa..e18ba287e8683 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 @@ -807,13 +807,11 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { test("throw errors for non-aggregate attributes with aggregation") { def checkAggregation(query: String, isInvalidQuery: Boolean = true) { - val logicalPlan = sql(query).queryExecution.logical - 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" + logicalPlan) + "Non-aggregate attribute(s) not detected\n") } else { // Should not throw sql(query).queryExecution.analyzed @@ -821,7 +819,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } checkAggregation("SELECT key, COUNT(*) FROM testData") - checkAggregation("SELECT COUNT(key), COUNT(*) FROM testData", false) + checkAggregation("SELECT COUNT(key), COUNT(*) FROM testData", isInvalidQuery = false) checkAggregation("SELECT value, COUNT(*) FROM testData GROUP BY key") checkAggregation("SELECT COUNT(value), SUM(key) FROM testData GROUP BY key", false) 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 e78145f4dda5a..ff91a0eb42049 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 @@ -24,7 +24,7 @@ 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.types._ import org.apache.spark.sql.test.TestSQLContext -import org.apache.spark.sql.{DataFrame, QueryTest, SQLConf} +import org.apache.spark.sql.{Column, DataFrame, QueryTest, SQLConf} /** * A test suite that tests Parquet filter2 API based filter pushdown optimization. @@ -51,8 +51,8 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED -> "true") { val query = rdd - .select(output.map(e => new org.apache.spark.sql.Column(e)): _*) - .where(new org.apache.spark.sql.Column(predicate)) + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) val maybeAnalyzedPredicate = query.queryExecution.executedPlan.collect { case plan: ParquetTableScan => plan.columnPruningPred 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 b746942cb1067..5efc3b1e30774 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 @@ -72,7 +72,8 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { if (conf.dialect == "sql") { super.sql(substituted) } else if (conf.dialect == "hiveql") { - new DataFrame(this, ddlParser(sqlText, false).getOrElse(HiveQl.parseSql(substituted))) + DataFrame(this, + ddlParser(sqlText, exceptionOnError = false).getOrElse(HiveQl.parseSql(substituted))) } else { sys.error(s"Unsupported SQL dialect: ${conf.dialect}. Try 'sql' or 'hiveql'") } 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 83244ce1e372a..fa997288a2848 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,10 +17,12 @@ 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.{Column, DataFrame, SQLContext, Strategy} +import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate @@ -29,7 +31,6 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand} import org.apache.spark.sql.execution._ -import org.apache.spark.sql.hive import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.sql.sources.CreateTableUsing @@ -56,14 +57,14 @@ private[hive] trait HiveStrategies { @Experimental object ParquetConversion extends Strategy { implicit class LogicalPlanHacks(s: DataFrame) { - def lowerCase = new DataFrame(s.sqlContext, s.logicalPlan) + def lowerCase = DataFrame(s.sqlContext, s.logicalPlan) def addPartitioningAttributes(attrs: Seq[Attribute]) = { // 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 } else { - new DataFrame( + DataFrame( s.sqlContext, s.logicalPlan transform { case p: ParquetRelation => p.copy(partitioningAttributes = attrs) @@ -96,13 +97,13 @@ private[hive] trait HiveStrategies { // We are going to throw the predicates and projection back at the whole optimization // sequence so lets unresolve all the attributes, allowing them to be rebound to the // matching parquet attributes. - val unresolvedOtherPredicates = new Column(otherPredicates.map(_ transform { + val unresolvedOtherPredicates = Column(otherPredicates.map(_ transform { case a: AttributeReference => UnresolvedAttribute(a.name) }).reduceOption(And).getOrElse(Literal(true))) val unresolvedProjection: Seq[Column] = projectList.map(_ transform { case a: AttributeReference => UnresolvedAttribute(a.name) - }).map(new Column(_)) + }).map(Column(_)) try { if (relation.hiveQlTable.isPartitioned) { From 0561c4544967fb853419f32e014fac9b8879b0db Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 2 Feb 2015 19:16:27 -0800 Subject: [PATCH 024/578] [SPARK-5154] [PySpark] [Streaming] Kafka streaming support in Python This PR brings the Python API for Spark Streaming Kafka data source. ``` class KafkaUtils(__builtin__.object) | Static methods defined here: | | createStream(ssc, zkQuorum, groupId, topics, storageLevel=StorageLevel(True, True, False, False, 2), keyDecoder=, valueDecoder=) | Create an input stream that pulls messages from a Kafka Broker. | | :param ssc: StreamingContext object | :param zkQuorum: Zookeeper quorum (hostname:port,hostname:port,..). | :param groupId: The group id for this consumer. | :param topics: Dict of (topic_name -> numPartitions) to consume. | Each partition is consumed in its own thread. | :param storageLevel: RDD storage level. | :param keyDecoder: A function used to decode key | :param valueDecoder: A function used to decode value | :return: A DStream object ``` run the example: ``` bin/spark-submit --driver-class-path external/kafka-assembly/target/scala-*/spark-streaming-kafka-assembly-*.jar examples/src/main/python/streaming/kafka_wordcount.py localhost:2181 test ``` Author: Davies Liu Author: Tathagata Das Closes #3715 from davies/kafka and squashes the following commits: d93bfe0 [Davies Liu] Update make-distribution.sh 4280d04 [Davies Liu] address comments e6d0427 [Davies Liu] Merge branch 'master' of github.com:apache/spark into kafka f257071 [Davies Liu] add tests for null in RDD 23b039a [Davies Liu] address comments 9af51c4 [Davies Liu] Merge branch 'kafka' of github.com:davies/spark into kafka a74da87 [Davies Liu] address comments dc1eed0 [Davies Liu] Update kafka_wordcount.py 31e2317 [Davies Liu] Update kafka_wordcount.py 370ba61 [Davies Liu] Update kafka.py 97386b3 [Davies Liu] address comment 2c567a5 [Davies Liu] update logging and comment 33730d1 [Davies Liu] Merge branch 'master' of github.com:apache/spark into kafka adeeb38 [Davies Liu] Merge pull request #3 from tdas/kafka-python-api aea8953 [Tathagata Das] Kafka-assembly for Python API eea16a7 [Davies Liu] refactor f6ce899 [Davies Liu] add example and fix bugs 98c8d17 [Davies Liu] fix python style 5697a01 [Davies Liu] bypass decoder in scala 048dbe6 [Davies Liu] fix python style 75d485e [Davies Liu] add mqtt 07923c4 [Davies Liu] support kafka in Python --- .../apache/spark/api/python/PythonRDD.scala | 66 ++++------- .../apache/spark/api/python/PythonUtils.scala | 5 + .../spark/api/python/PythonRDDSuite.scala | 23 +++- .../main/python/streaming/kafka_wordcount.py | 54 +++++++++ external/kafka-assembly/pom.xml | 106 ++++++++++++++++++ pom.xml | 1 + project/SparkBuild.scala | 14 ++- python/pyspark/serializers.py | 9 +- python/pyspark/streaming/kafka.py | 83 ++++++++++++++ python/pyspark/tests.py | 10 +- 10 files changed, 313 insertions(+), 58 deletions(-) create mode 100644 examples/src/main/python/streaming/kafka_wordcount.py create mode 100644 external/kafka-assembly/pom.xml create mode 100644 python/pyspark/streaming/kafka.py 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 119e0459c5d1b..b89effc16d36d 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 @@ -316,6 +316,7 @@ private object SpecialLengths { val PYTHON_EXCEPTION_THROWN = -2 val TIMING_DATA = -3 val END_OF_STREAM = -4 + val NULL = -5 } private[spark] object PythonRDD extends Logging { @@ -374,54 +375,25 @@ private[spark] object PythonRDD extends Logging { } def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) { - // The right way to implement this would be to use TypeTags to get the full - // type of T. Since I don't want to introduce breaking changes throughout the - // entire Spark API, I have to use this hacky approach: - if (iter.hasNext) { - val first = iter.next() - val newIter = Seq(first).iterator ++ iter - first match { - case arr: Array[Byte] => - newIter.asInstanceOf[Iterator[Array[Byte]]].foreach { bytes => - dataOut.writeInt(bytes.length) - dataOut.write(bytes) - } - case string: String => - newIter.asInstanceOf[Iterator[String]].foreach { str => - writeUTF(str, dataOut) - } - case stream: PortableDataStream => - newIter.asInstanceOf[Iterator[PortableDataStream]].foreach { stream => - val bytes = stream.toArray() - dataOut.writeInt(bytes.length) - dataOut.write(bytes) - } - case (key: String, stream: PortableDataStream) => - newIter.asInstanceOf[Iterator[(String, PortableDataStream)]].foreach { - case (key, stream) => - writeUTF(key, dataOut) - val bytes = stream.toArray() - dataOut.writeInt(bytes.length) - dataOut.write(bytes) - } - case (key: String, value: String) => - newIter.asInstanceOf[Iterator[(String, String)]].foreach { - case (key, value) => - writeUTF(key, dataOut) - writeUTF(value, dataOut) - } - case (key: Array[Byte], value: Array[Byte]) => - newIter.asInstanceOf[Iterator[(Array[Byte], Array[Byte])]].foreach { - case (key, value) => - dataOut.writeInt(key.length) - dataOut.write(key) - dataOut.writeInt(value.length) - dataOut.write(value) - } - case other => - throw new SparkException("Unexpected element type " + first.getClass) - } + + def write(obj: Any): Unit = obj match { + case null => + dataOut.writeInt(SpecialLengths.NULL) + case arr: Array[Byte] => + dataOut.writeInt(arr.length) + dataOut.write(arr) + case str: String => + writeUTF(str, dataOut) + case stream: PortableDataStream => + write(stream.toArray()) + case (key, value) => + write(key) + write(value) + case other => + throw new SparkException("Unexpected element type " + other.getClass) } + + iter.foreach(write) } /** diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index be5ebfa9219d3..b7cfc8bd9c542 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -22,6 +22,7 @@ import java.io.{File, InputStream, IOException, OutputStream} import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkContext +import org.apache.spark.api.java.{JavaSparkContext, JavaRDD} private[spark] object PythonUtils { /** Get the PYTHONPATH for PySpark, either from SPARK_HOME, if it is set, or from our JAR */ @@ -39,4 +40,8 @@ private[spark] object PythonUtils { def mergePythonPaths(paths: String*): String = { paths.filter(_ != "").mkString(File.pathSeparator) } + + def generateRDDWithNull(sc: JavaSparkContext): JavaRDD[String] = { + sc.parallelize(List("a", null, "b")) + } } diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala index 7b866f08a0e9f..c63d834f9048b 100644 --- a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala @@ -23,11 +23,22 @@ import org.scalatest.FunSuite class PythonRDDSuite extends FunSuite { - test("Writing large strings to the worker") { - val input: List[String] = List("a"*100000) - val buffer = new DataOutputStream(new ByteArrayOutputStream) - PythonRDD.writeIteratorToStream(input.iterator, buffer) - } + test("Writing large strings to the worker") { + val input: List[String] = List("a"*100000) + val buffer = new DataOutputStream(new ByteArrayOutputStream) + PythonRDD.writeIteratorToStream(input.iterator, buffer) + } + test("Handle nulls gracefully") { + val buffer = new DataOutputStream(new ByteArrayOutputStream) + // Should not have NPE when write an Iterator with null in it + // The correctness will be tested in Python + PythonRDD.writeIteratorToStream(Iterator("a", null), buffer) + PythonRDD.writeIteratorToStream(Iterator(null, "a"), buffer) + PythonRDD.writeIteratorToStream(Iterator("a".getBytes, null), buffer) + PythonRDD.writeIteratorToStream(Iterator(null, "a".getBytes), buffer) + PythonRDD.writeIteratorToStream(Iterator((null, null), ("a", null), (null, "b")), buffer) + PythonRDD.writeIteratorToStream( + Iterator((null, null), ("a".getBytes, null), (null, "b".getBytes)), buffer) + } } - diff --git a/examples/src/main/python/streaming/kafka_wordcount.py b/examples/src/main/python/streaming/kafka_wordcount.py new file mode 100644 index 0000000000000..ed398a82b8bb0 --- /dev/null +++ b/examples/src/main/python/streaming/kafka_wordcount.py @@ -0,0 +1,54 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" + Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + Usage: network_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 + + and then run the example + `$ bin/spark-submit --driver-class-path external/kafka-assembly/target/scala-*/\ + spark-streaming-kafka-assembly-*.jar examples/src/main/python/streaming/kafka_wordcount.py \ + localhost:2181 test` +""" + +import sys + +from pyspark import SparkContext +from pyspark.streaming import StreamingContext +from pyspark.streaming.kafka import KafkaUtils + +if __name__ == "__main__": + if len(sys.argv) != 3: + print >> sys.stderr, "Usage: kafka_wordcount.py " + exit(-1) + + sc = SparkContext(appName="PythonStreamingKafkaWordCount") + ssc = StreamingContext(sc, 1) + + zkQuorum, topic = sys.argv[1:] + kvs = KafkaUtils.createStream(ssc, zkQuorum, "spark-streaming-consumer", {topic: 1}) + lines = kvs.map(lambda x: x[1]) + counts = lines.flatMap(lambda line: line.split(" ")) \ + .map(lambda word: (word, 1)) \ + .reduceByKey(lambda a, b: a+b) + counts.pprint() + + ssc.start() + ssc.awaitTermination() diff --git a/external/kafka-assembly/pom.xml b/external/kafka-assembly/pom.xml new file mode 100644 index 0000000000000..503fc129dc4f2 --- /dev/null +++ b/external/kafka-assembly/pom.xml @@ -0,0 +1,106 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.3.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-streaming-kafka-assembly_2.10 + jar + Spark Project External Kafka Assembly + http://spark.apache.org/ + + + streaming-kafka-assembly + scala-${scala.binary.version} + spark-streaming-kafka-assembly-${project.version}.jar + ${project.build.directory}/${spark.jar.dir}/${spark.jar.basename} + + + + + org.apache.spark + spark-streaming-kafka_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-shade-plugin + + false + ${spark.jar} + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + log4j.properties + + + + + + + + + + + + diff --git a/pom.xml b/pom.xml index d4112b03d9ee4..08d1cc33e4701 100644 --- a/pom.xml +++ b/pom.xml @@ -1629,6 +1629,7 @@ external/kafka + external/kafka-assembly diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ded4b5443a904..fbc8983b953b7 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -44,8 +44,9 @@ object BuildCommons { sparkKinesisAsl) = Seq("yarn", "yarn-stable", "java8-tests", "ganglia-lgpl", "kinesis-asl").map(ProjectRef(buildLocation, _)) - val assemblyProjects@Seq(assembly, examples, networkYarn) = - Seq("assembly", "examples", "network-yarn").map(ProjectRef(buildLocation, _)) + val assemblyProjects@Seq(assembly, examples, networkYarn, streamingKafkaAssembly) = + Seq("assembly", "examples", "network-yarn", "streaming-kafka-assembly") + .map(ProjectRef(buildLocation, _)) val tools = ProjectRef(buildLocation, "tools") // Root project. @@ -300,7 +301,14 @@ object Assembly { sys.props.get("hadoop.version") .getOrElse(SbtPomKeys.effectivePom.value.getProperties.get("hadoop.version").asInstanceOf[String]) }, - jarName in assembly := s"${moduleName.value}-${version.value}-hadoop${hadoopVersion.value}.jar", + jarName in assembly <<= (version, moduleName, hadoopVersion) map { (v, mName, hv) => + if (mName.contains("streaming-kafka-assembly")) { + // This must match the same name used in maven (see external/kafka-assembly/pom.xml) + s"${mName}-${v}.jar" + } else { + s"${mName}-${v}-hadoop${hv}.jar" + } + }, mergeStrategy in assembly := { case PathList("org", "datanucleus", xs @ _*) => MergeStrategy.discard case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index b8bda835174b2..0ffb41d02f6f6 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -70,6 +70,7 @@ class SpecialLengths(object): PYTHON_EXCEPTION_THROWN = -2 TIMING_DATA = -3 END_OF_STREAM = -4 + NULL = -5 class Serializer(object): @@ -133,6 +134,8 @@ def load_stream(self, stream): def _write_with_length(self, obj, stream): serialized = self.dumps(obj) + if serialized is None: + raise ValueError("serialized value should not be None") if len(serialized) > (1 << 31): raise ValueError("can not serialize object larger than 2G") write_int(len(serialized), stream) @@ -145,8 +148,10 @@ def _read_with_length(self, stream): length = read_int(stream) if length == SpecialLengths.END_OF_DATA_SECTION: raise EOFError + elif length == SpecialLengths.NULL: + return None obj = stream.read(length) - if obj == "": + if len(obj) < length: raise EOFError return self.loads(obj) @@ -484,6 +489,8 @@ def loads(self, stream): length = read_int(stream) if length == SpecialLengths.END_OF_DATA_SECTION: raise EOFError + elif length == SpecialLengths.NULL: + return None s = stream.read(length) return s.decode("utf-8") if self.use_unicode else s diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py new file mode 100644 index 0000000000000..19ad71f99d4d5 --- /dev/null +++ b/python/pyspark/streaming/kafka.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 py4j.java_collections import MapConverter +from py4j.java_gateway import java_import, Py4JError + +from pyspark.storagelevel import StorageLevel +from pyspark.serializers import PairDeserializer, NoOpSerializer +from pyspark.streaming import DStream + +__all__ = ['KafkaUtils', 'utf8_decoder'] + + +def utf8_decoder(s): + """ Decode the unicode as UTF-8 """ + return s and s.decode('utf-8') + + +class KafkaUtils(object): + + @staticmethod + def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={}, + storageLevel=StorageLevel.MEMORY_AND_DISK_SER_2, + keyDecoder=utf8_decoder, valueDecoder=utf8_decoder): + """ + Create an input stream that pulls messages from a Kafka Broker. + + :param ssc: StreamingContext object + :param zkQuorum: Zookeeper quorum (hostname:port,hostname:port,..). + :param groupId: The group id for this consumer. + :param topics: Dict of (topic_name -> numPartitions) to consume. + Each partition is consumed in its own thread. + :param kafkaParams: Additional params for Kafka + :param storageLevel: RDD storage level. + :param keyDecoder: A function used to decode key (default is utf8_decoder) + :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, + "zookeeper.connection.timeout.ms": "10000", + }) + if not isinstance(topics, dict): + raise TypeError("topics should be dict") + jtopics = MapConverter().convert(topics, ssc.sparkContext._gateway._gateway_client) + 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: + # 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" + raise e + ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) + stream = DStream(jstream, ssc, ser) + return stream.map(lambda (k, v): (keyDecoder(k), valueDecoder(v))) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index fef6c92875a1c..c7d0622d65f25 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -47,9 +47,10 @@ from pyspark.conf import SparkConf from pyspark.context import SparkContext +from pyspark.rdd import RDD from pyspark.files import SparkFiles from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \ - CloudPickleSerializer, CompressedSerializer + CloudPickleSerializer, CompressedSerializer, UTF8Deserializer, NoOpSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter from pyspark.sql import SQLContext, IntegerType, Row, ArrayType, StructType, StructField, \ UserDefinedType, DoubleType @@ -716,6 +717,13 @@ def test_sample(self): wr_s21 = rdd.sample(True, 0.4, 21).collect() self.assertNotEqual(set(wr_s11), set(wr_s21)) + def test_null_in_rdd(self): + jrdd = self.sc._jvm.PythonUtils.generateRDDWithNull(self.sc._jsc) + 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()) + def test_multiple_python_java_RDD_conversions(self): # Regression test for SPARK-5361 data = [ From 1bcd46574e442e20f55709d70573f271ce44e5b9 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 2 Feb 2015 19:34:25 -0800 Subject: [PATCH 025/578] [SPARK-5512][Mllib] Run the PIC algorithm with initial vector suggected by the PIC paper As suggested by the paper of Power Iteration Clustering, it is useful to set the initial vector v0 as the degree vector d. This pr tries to add a running method for that. Author: Liang-Chi Hsieh Closes #4301 from viirya/pic_degreevector and squashes the following commits: 7db28fb [Liang-Chi Hsieh] Refactor it to address comments. 19cf94e [Liang-Chi Hsieh] Add an option to select initialization method. ec88567 [Liang-Chi Hsieh] Run the PIC algorithm with degree vector d as suggected by the PIC paper. --- .../clustering/PowerIterationClustering.scala | 41 +++++++++++++++++-- .../PowerIterationClusteringSuite.scala | 10 +++++ 2 files changed, 47 insertions(+), 4 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 fcb9a3643cc48..9b5c155b0a805 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 @@ -43,15 +43,19 @@ class PowerIterationClusteringModel( * * @param k Number of clusters. * @param maxIterations Maximum number of iterations of the PIC algorithm. + * @param initMode Initialization mode. */ class PowerIterationClustering private[clustering] ( private var k: Int, - private var maxIterations: Int) extends Serializable { + private var maxIterations: Int, + private var initMode: String) extends Serializable { import org.apache.spark.mllib.clustering.PowerIterationClustering._ - /** Constructs a PIC instance with default parameters: {k: 2, maxIterations: 100}. */ - def this() = this(k = 2, maxIterations = 100) + /** Constructs a PIC instance with default parameters: {k: 2, maxIterations: 100, + * initMode: "random"}. + */ + def this() = this(k = 2, maxIterations = 100, initMode = "random") /** * Set the number of clusters. @@ -69,6 +73,18 @@ class PowerIterationClustering private[clustering] ( this } + /** + * Set the initialization mode. This can be either "random" to use a random vector + * as vertex properties, or "degree" to use normalized sum similarities. Default: random. + */ + def setInitializationMode(mode: String): this.type = { + this.initMode = mode match { + case "random" | "degree" => mode + case _ => throw new IllegalArgumentException("Invalid initialization mode: " + mode) + } + this + } + /** * Run the PIC algorithm. * @@ -82,7 +98,10 @@ class PowerIterationClustering private[clustering] ( */ def run(similarities: RDD[(Long, Long, Double)]): PowerIterationClusteringModel = { val w = normalize(similarities) - val w0 = randomInit(w) + val w0 = initMode match { + case "random" => randomInit(w) + case "degree" => initDegreeVector(w) + } pic(w0) } @@ -148,6 +167,20 @@ private[clustering] object PowerIterationClustering extends Logging { GraphImpl.fromExistingRDDs(VertexRDD(v0), g.edges) } + /** + * Generates the degree vector as the vertex properties (v0) to start power iteration. + * It is not exactly the node degrees but just the normalized sum similarities. Call it + * as degree vector because it is used in the PIC paper. + * + * @param g a graph representing the normalized affinity matrix (W) + * @return a graph with edges representing W and vertices representing the degree vector + */ + def initDegreeVector(g: Graph[Double, Double]): Graph[Double, Double] = { + val sum = g.vertices.values.sum() + val v0 = g.vertices.mapValues(_ / sum) + GraphImpl.fromExistingRDDs(VertexRDD(v0), g.edges) + } + /** * Runs power iteration. * @param g input graph with edges representing the normalized affinity matrix (W) and vertices 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 2bae465d392aa..03ecd9ca730be 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 @@ -55,6 +55,16 @@ class PowerIterationClusteringSuite extends FunSuite with MLlibTestSparkContext predictions(c) += i } assert(predictions.toSet == Set((0 to 3).toSet, (4 to 15).toSet)) + + val model2 = new PowerIterationClustering() + .setK(2) + .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 + } + assert(predictions2.toSet == Set((0 to 3).toSet, (4 to 15).toSet)) } test("normalize and powerIter") { From 8f471a66db0571a76a21c0d93312197fee16174a Mon Sep 17 00:00:00 2001 From: Tor Myklebust Date: Mon, 2 Feb 2015 19:50:14 -0800 Subject: [PATCH 026/578] [SPARK-5472][SQL] A JDBC data source for Spark SQL. This pull request contains a Spark SQL data source that can pull data from, and can put data into, a JDBC database. I have tested both read and write support with H2, MySQL, and Postgres. It would surprise me if both read and write support worked flawlessly out-of-the-box for any other database; different databases have different names for different JDBC data types and different meanings for SQL types with the same name. However, this code is designed (see `DriverQuirks.scala`) to make it *relatively* painless to add support for another database by augmenting the type mapping contained in this PR. Author: Tor Myklebust Closes #4261 from tmyklebu/master and squashes the following commits: cf167ce [Tor Myklebust] Work around other Java tests ruining TestSQLContext. 67893bf [Tor Myklebust] Move the jdbcRDD methods into SQLContext itself. 585f95b [Tor Myklebust] Dependencies go into the project's pom.xml. 829d5ba [Tor Myklebust] Merge branch 'master' of https://github.com/apache/spark 41647ef [Tor Myklebust] Hide a couple things that don't need to be public. 7318aea [Tor Myklebust] Fix scalastyle warnings. a09eeac [Tor Myklebust] JDBC data source for Spark SQL. 176bb98 [Tor Myklebust] Add test deps for JDBC support. --- sql/core/pom.xml | 24 + .../org/apache/spark/sql/jdbc/JDBCUtils.java | 59 +++ .../org/apache/spark/sql/SQLContext.scala | 49 +- .../apache/spark/sql/jdbc/DriverQuirks.scala | 99 +++++ .../org/apache/spark/sql/jdbc/JDBCRDD.scala | 417 ++++++++++++++++++ .../apache/spark/sql/jdbc/JDBCRelation.scala | 133 ++++++ .../spark/sql/jdbc/JavaJDBCTrampoline.scala | 30 ++ .../org/apache/spark/sql/jdbc/jdbc.scala | 235 ++++++++++ .../apache/spark/sql/jdbc/JavaJDBCTest.java | 102 +++++ .../apache/spark/sql/jdbc/DockerHacks.scala | 51 +++ .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 248 +++++++++++ .../spark/sql/jdbc/JDBCWriteSuite.scala | 107 +++++ .../spark/sql/jdbc/MySQLIntegration.scala | 235 ++++++++++ .../spark/sql/jdbc/PostgresIntegration.scala | 149 +++++++ 14 files changed, 1937 insertions(+), 1 deletion(-) create 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/jdbc/DriverQuirks.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/jdbc/JavaJDBCTrampoline.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala create mode 100644 sql/core/src/test/java/org/apache/spark/sql/jdbc/JavaJDBCTest.java create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/jdbc/DockerHacks.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala create 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 3e9ef07df9db6..1a0c77d282307 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -76,6 +76,30 @@ scalacheck_${scala.binary.version} test + + com.h2database + h2 + 1.4.183 + test + + + mysql + mysql-connector-java + 5.1.34 + test + + + org.postgresql + postgresql + 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/main/java/org/apache/spark/sql/jdbc/JDBCUtils.java b/sql/core/src/main/java/org/apache/spark/sql/jdbc/JDBCUtils.java new file mode 100644 index 0000000000000..aa441b2096f18 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/jdbc/JDBCUtils.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index d0bbb5f7a34f1..f4692b3ff59d3 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 @@ -24,7 +24,7 @@ import scala.collection.immutable import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.SparkContext +import org.apache.spark.{SparkContext, Partition} import org.apache.spark.annotation.{AlphaComponent, DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaSparkContext, JavaRDD} import org.apache.spark.rdd.RDD @@ -36,6 +36,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution._ import org.apache.spark.sql.json._ +import org.apache.spark.sql.jdbc.{JDBCPartition, JDBCPartitioningInfo, JDBCRelation} import org.apache.spark.sql.sources.{LogicalRelation, BaseRelation, DDLParser, DataSourceStrategy} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -334,6 +335,52 @@ class SQLContext(@transient val sparkContext: SparkContext) applySchema(rowRDD, appliedSchema) } + /** + * :: Experimental :: + * Construct an RDD representing the database table accessible via JDBC URL + * url named table. + */ + @Experimental + def jdbcRDD(url: String, table: String): DataFrame = { + jdbcRDD(url, table, null.asInstanceOf[JDBCPartitioningInfo]) + } + + /** + * :: 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. + */ + @Experimental + def jdbcRDD(url: String, table: String, partitioning: JDBCPartitioningInfo): + DataFrame = { + val parts = JDBCRelation.columnPartition(partitioning) + jdbcRDD(url, table, parts) + } + + /** + * :: Experimental :: + * Construct an RDD 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. + */ + @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]) + jdbcRDD(url, table, parts) + } + + 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 * during the lifetime of this instance of SQLContext. 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 new file mode 100644 index 0000000000000..1704be7fcbd30 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/DriverQuirks.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.sql.types._ + +import java.sql.Types + + +/** + * Encapsulates workarounds for the extensions, quirks, and bugs in various + * databases. Lots of databases define types that aren't explicitly supported + * by the JDBC spec. Some JDBC drivers also report inaccurate + * information---for instance, BIT(n>1) being reported as a BIT type is quite + * common, even though BIT in JDBC is meant for single-bit values. Also, there + * does not appear to be a standard name for an unbounded string or binary + * type; we use BLOB and CLOB by default but override with database-specific + * alternatives when these are absent or do not behave correctly. + * + * Currently, the only thing DriverQuirks does is handle type mapping. + * `getCatalystType` is used when reading from a JDBC table and `getJDBCType` + * is used when writing to a JDBC table. If `getCatalystType` returns `null`, + * the default type handling is used for the given JDBC type. Similarly, + * if `getJDBCType` returns `(null, None)`, the default type handling is used + * for the given Catalyst type. + */ +private[sql] abstract class DriverQuirks { + def getCatalystType(sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): DataType + def getJDBCType(dt: DataType): (String, Option[Int]) +} + +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")) { + new MySQLQuirks() + } else if (url.substring(0, 15).equals("jdbc:postgresql")) { + new PostgresQuirks() + } else { + new NoQuirks() + } + } +} + +private[sql] class NoQuirks extends DriverQuirks { + def getCatalystType(sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): DataType = + null + def getJDBCType(dt: DataType): (String, Option[Int]) = (null, None) +} + +private[sql] class PostgresQuirks extends DriverQuirks { + def getCatalystType(sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): DataType = { + if (sqlType == Types.BIT && typeName.equals("bit") && size != 1) { + BinaryType + } else if (sqlType == Types.OTHER && typeName.equals("cidr")) { + StringType + } else if (sqlType == Types.OTHER && typeName.equals("inet")) { + StringType + } else null + } + + def getJDBCType(dt: DataType): (String, Option[Int]) = dt match { + case StringType => ("TEXT", Some(java.sql.Types.CHAR)) + case BinaryType => ("BYTEA", Some(java.sql.Types.BINARY)) + case BooleanType => ("BOOLEAN", Some(java.sql.Types.BOOLEAN)) + case _ => (null, None) + } +} + +private[sql] class MySQLQuirks extends DriverQuirks { + def getCatalystType(sqlType: Int, typeName: String, size: Int, md: MetadataBuilder): DataType = { + if (sqlType == Types.VARBINARY && typeName.equals("BIT") && size != 1) { + // This could instead be a BinaryType if we'd rather return bit-vectors of up to 64 bits as + // byte arrays instead of longs. + md.putLong("binarylong", 1) + LongType + } else if (sqlType == Types.BIT && typeName.equals("TINYINT")) { + BooleanType + } else null + } + def getJDBCType(dt: DataType): (String, Option[Int]) = (null, None) +} 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 new file mode 100644 index 0000000000000..a2f94675fb5a3 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -0,0 +1,417 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.{Connection, DatabaseMetaData, DriverManager, ResultSet, ResultSetMetaData, SQLException} +import scala.collection.mutable.ArrayBuffer + +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._ + +private[sql] object JDBCRDD extends Logging { + /** + * Maps a JDBC type to a Catalyst type. This function is called only when + * the DriverQuirks class corresponding to your database driver returns null. + * + * @param sqlType - A field of java.sql.Types + * @return The Catalyst type corresponding to sqlType. + */ + private def getCatalystType(sqlType: Int): DataType = { + val answer = sqlType match { + case java.sql.Types.ARRAY => null + case java.sql.Types.BIGINT => LongType + case java.sql.Types.BINARY => BinaryType + case java.sql.Types.BIT => BooleanType // Per JDBC; Quirks handles quirky drivers. + case java.sql.Types.BLOB => BinaryType + case java.sql.Types.BOOLEAN => BooleanType + case java.sql.Types.CHAR => StringType + case java.sql.Types.CLOB => StringType + case java.sql.Types.DATALINK => null + case java.sql.Types.DATE => DateType + case java.sql.Types.DECIMAL => DecimalType.Unlimited + case java.sql.Types.DISTINCT => null + case java.sql.Types.DOUBLE => DoubleType + case java.sql.Types.FLOAT => FloatType + case java.sql.Types.INTEGER => IntegerType + case java.sql.Types.JAVA_OBJECT => null + case java.sql.Types.LONGNVARCHAR => StringType + case java.sql.Types.LONGVARBINARY => BinaryType + case java.sql.Types.LONGVARCHAR => StringType + case java.sql.Types.NCHAR => StringType + case java.sql.Types.NCLOB => StringType + case java.sql.Types.NULL => null + case java.sql.Types.NUMERIC => DecimalType.Unlimited + case java.sql.Types.OTHER => null + case java.sql.Types.REAL => DoubleType + case java.sql.Types.REF => StringType + case java.sql.Types.ROWID => LongType + case java.sql.Types.SMALLINT => IntegerType + case java.sql.Types.SQLXML => StringType + case java.sql.Types.STRUCT => StringType + case java.sql.Types.TIME => TimestampType + case java.sql.Types.TIMESTAMP => TimestampType + case java.sql.Types.TINYINT => IntegerType + case java.sql.Types.VARBINARY => BinaryType + case java.sql.Types.VARCHAR => StringType + case _ => null + } + + if (answer == null) throw new SQLException("Unsupported type " + sqlType) + answer + } + + /** + * Takes a (schema, table) specification and returns the table's Catalyst + * schema. + * + * @param url - The JDBC url to fetch information from. + * @param table - The table name of the desired table. This may also be a + * SQL query wrapped in parentheses. + * + * @return A StructType giving the table's Catalyst schema. + * @throws SQLException if the table specification is garbage. + * @throws SQLException if the table contains an unsupported type. + */ + def resolveTable(url: String, table: String): StructType = { + val quirks = DriverQuirks.get(url) + val conn: Connection = DriverManager.getConnection(url) + try { + val rs = conn.prepareStatement(s"SELECT * FROM $table WHERE 1=0").executeQuery() + try { + val rsmd = rs.getMetaData + val ncols = rsmd.getColumnCount + var fields = new Array[StructField](ncols); + var i = 0 + while (i < ncols) { + val columnName = rsmd.getColumnName(i + 1) + val dataType = rsmd.getColumnType(i + 1) + val typeName = rsmd.getColumnTypeName(i + 1) + val fieldSize = rsmd.getPrecision(i + 1) + val nullable = rsmd.isNullable(i + 1) != ResultSetMetaData.columnNoNulls + val metadata = new MetadataBuilder().putString("name", columnName) + var columnType = quirks.getCatalystType(dataType, typeName, fieldSize, metadata) + if (columnType == null) columnType = getCatalystType(dataType) + fields(i) = StructField(columnName, columnType, nullable, metadata.build()) + i = i + 1 + } + return new StructType(fields) + } finally { + rs.close() + } + } finally { + conn.close() + } + + throw new RuntimeException("This line is unreachable.") + } + + /** + * Prune all but the specified columns from the specified Catalyst schema. + * + * @param schema - The Catalyst schema of the master table + * @param columns - The list of desired columns + * + * @return A Catalyst schema corresponding to columns in the given order. + */ + private def pruneSchema(schema: StructType, columns: Array[String]): StructType = { + val fieldMap = Map(schema.fields map { x => x.metadata.getString("name") -> x }: _*) + new StructType(columns map { name => fieldMap(name) }) + } + + /** + * Given a driver string and an url, return a function that loads the + * specified driver string then returns a connection to the JDBC url. + * getConnector is run on the driver code, while the function it returns + * is run on the executor. + * + * @param driver - The class name of the JDBC driver for the given url. + * @param url - The JDBC url to connect to. + * + * @return A function that loads the driver and connects to the url. + */ + def getConnector(driver: String, url: String): () => Connection = { + () => { + try { + if (driver != null) Class.forName(driver) + } catch { + case e: ClassNotFoundException => { + logWarning(s"Couldn't find class $driver", e); + } + } + DriverManager.getConnection(url) + } + } + /** + * Build and return JDBCRDD from the given information. + * + * @param sc - Your SparkContext. + * @param schema - The Catalyst schema of the underlying database table. + * @param driver - The class name of the JDBC driver for the given url. + * @param url - The JDBC url to connect to. + * @param fqTable - The fully-qualified table name (or paren'd SQL query) to use. + * @param requiredColumns - The names of the columns to SELECT. + * @param filters - The filters to include in all WHERE clauses. + * @param parts - An array of JDBCPartitions specifying partition ids and + * per-partition WHERE clauses. + * + * @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] = { + val prunedSchema = pruneSchema(schema, requiredColumns) + + return new JDBCRDD(sc, + getConnector(driver, url), + prunedSchema, + fqTable, + requiredColumns, + filters, + parts) + } +} + +/** + * An RDD representing a table in a database accessed via JDBC. Both the + * driver code and the workers must be able to access the database; the driver + * needs to fetch the schema while the workers need to fetch the data. + */ +private[sql] class JDBCRDD( + sc: SparkContext, + getConnection: () => Connection, + schema: StructType, + fqTable: String, + columns: Array[String], + filters: Array[Filter], + partitions: Array[Partition]) + extends RDD[Row](sc, Nil) { + + /** + * Retrieve the list of partitions corresponding to this RDD. + */ + override def getPartitions: Array[Partition] = partitions + + /** + * `columns`, but as a String suitable for injection into a SQL query. + */ + private val columnList: String = { + val sb = new StringBuilder() + columns.foreach(x => sb.append(",").append(x)) + if (sb.length == 0) "1" else sb.substring(1) + } + + /** + * 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 _ => null + } + + /** + * `filters`, but as a WHERE clause suitable for injection into a SQL query. + */ + private val filterWhereClause: String = { + val filterStrings = filters map compileFilter filter (_ != null) + if (filterStrings.size > 0) { + val sb = new StringBuilder("WHERE ") + filterStrings.foreach(x => sb.append(x).append(" AND ")) + sb.substring(0, sb.length - 5) + } else "" + } + + /** + * A WHERE clause representing both `filters`, if any, and the current partition. + */ + private def getWhereClause(part: JDBCPartition): String = { + if (part.whereClause != null && filterWhereClause.length > 0) { + filterWhereClause + " AND " + part.whereClause + } else if (part.whereClause != null) { + "WHERE " + part.whereClause + } else { + filterWhereClause + } + } + + // Each JDBC-to-Catalyst conversion corresponds to a tag defined here so that + // we don't have to potentially poke around in the Metadata once for every + // row. + // Is there a better way to do this? I'd rather be using a type that + // contains only the tags I define. + abstract class JDBCConversion + case object BooleanConversion extends JDBCConversion + case object DateConversion extends JDBCConversion + case object DecimalConversion extends JDBCConversion + case object DoubleConversion extends JDBCConversion + case object FloatConversion extends JDBCConversion + case object IntegerConversion extends JDBCConversion + case object LongConversion extends JDBCConversion + case object BinaryLongConversion extends JDBCConversion + case object StringConversion extends JDBCConversion + case object TimestampConversion extends JDBCConversion + case object BinaryConversion extends JDBCConversion + + /** + * Maps a StructType to a type tag list. + */ + def getConversions(schema: StructType): Array[JDBCConversion] = { + schema.fields.map(sf => sf.dataType match { + case BooleanType => BooleanConversion + case DateType => DateConversion + case DecimalType.Unlimited => DecimalConversion + case DoubleType => DoubleConversion + case FloatType => FloatConversion + case IntegerType => IntegerConversion + case LongType => + if (sf.metadata.contains("binarylong")) BinaryLongConversion else LongConversion + case StringType => StringConversion + case TimestampType => TimestampConversion + case BinaryType => BinaryConversion + case _ => throw new IllegalArgumentException(s"Unsupported field $sf") + }).toArray + } + + + /** + * Runs the SQL query against the JDBC driver. + */ + override def compute(thePart: Partition, context: TaskContext) = new Iterator[Row] { + var closed = false + var finished = false + var gotNext = false + var nextValue: Row = null + + context.addTaskCompletionListener{ context => close() } + val part = thePart.asInstanceOf[JDBCPartition] + val conn = getConnection() + + // H2's JDBC driver does not support the setSchema() method. We pass a + // fully-qualified table name in the SELECT statement. I don't know how to + // talk about a table in a completely portable way. + + val myWhereClause = getWhereClause(part) + + val sqlText = s"SELECT $columnList FROM $fqTable $myWhereClause" + val stmt = conn.prepareStatement(sqlText, + ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY) + val rs = stmt.executeQuery() + + val conversions = getConversions(schema) + val mutableRow = new SpecificMutableRow(schema.fields.map(x => x.dataType)) + + def getNext(): Row = { + if (rs.next()) { + var i = 0 + while (i < conversions.length) { + val pos = i + 1 + conversions(i) match { + case BooleanConversion => mutableRow.setBoolean(i, rs.getBoolean(pos)) + 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)) + case StringConversion => mutableRow.setString(i, rs.getString(pos)) + case TimestampConversion => mutableRow.update(i, rs.getTimestamp(pos)) + case BinaryConversion => mutableRow.update(i, rs.getBytes(pos)) + case BinaryLongConversion => { + val bytes = rs.getBytes(pos) + var ans = 0L + var j = 0 + while (j < bytes.size) { + ans = 256*ans + (255 & bytes(j)) + j = j + 1; + } + mutableRow.setLong(i, ans) + } + } + if (rs.wasNull) mutableRow.setNullAt(i) + i = i + 1 + } + mutableRow + } else { + finished = true + null.asInstanceOf[Row] + } + } + + def close() { + if (closed) return + try { + if (null != rs && ! rs.isClosed()) { + rs.close() + } + } catch { + case e: Exception => logWarning("Exception closing resultset", e) + } + try { + if (null != stmt && ! stmt.isClosed()) { + stmt.close() + } + } catch { + case e: Exception => logWarning("Exception closing statement", e) + } + try { + if (null != conn && ! conn.isClosed()) { + conn.close() + } + logInfo("closed connection") + } catch { + case e: Exception => logWarning("Exception closing connection", e) + } + } + + override def hasNext: Boolean = { + if (!finished) { + if (!gotNext) { + nextValue = getNext() + if (finished) { + close() + } + gotNext = true + } + } + !finished + } + + override def next(): Row = { + if (!hasNext) { + throw new NoSuchElementException("End of stream") + } + 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 new file mode 100644 index 0000000000000..e09125e406ba2 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.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.sql.jdbc + +import scala.collection.mutable.ArrayBuffer +import java.sql.DriverManager + +import org.apache.spark.Partition +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.sources._ + +/** + * Data corresponding to one partition of a JDBCRDD. + */ +private[sql] case class JDBCPartition(whereClause: String, idx: Int) extends Partition { + override def index: Int = idx +} + +/** + * Instructions on how to partition the table among workers. + */ +private[sql] case class JDBCPartitioningInfo( + column: String, + lowerBound: Long, + upperBound: Long, + numPartitions: Int) + +private[sql] object JDBCRelation { + /** + * Given a partitioning schematic (a column of integral type, a number of + * partitions, and upper and lower bounds on the column's value), generate + * WHERE clauses for each partition so that each row in the table appears + * 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)) + + val numPartitions = partitioning.numPartitions + val column = partitioning.column + if (numPartitions == 1) return Array[Partition](JDBCPartition(null, 0)) + // Overflow and silliness can happen if you subtract then divide. + // Here we get a little roundoff, but that's (hopefully) OK. + val stride: Long = (partitioning.upperBound / numPartitions + - partitioning.lowerBound / numPartitions) + var i: Int = 0 + var currentValue: Long = partitioning.lowerBound + var ans = new ArrayBuffer[Partition]() + while (i < numPartitions) { + 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") + ans += JDBCPartition(whereClause, i) + i = i + 1 + } + ans.toArray + } +} + +private[sql] class DefaultSource extends RelationProvider { + /** Returns a new base relation with the given parameters. */ + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + val url = parameters.getOrElse("url", sys.error("Option 'url' not specified")) + val driver = parameters.getOrElse("driver", null) + val table = parameters.getOrElse("dbtable", sys.error("Option 'dbtable' not specified")) + val partitionColumn = parameters.getOrElse("partitionColumn", null) + val lowerBound = parameters.getOrElse("lowerBound", null) + val upperBound = parameters.getOrElse("upperBound", null) + val numPartitions = parameters.getOrElse("numPartitions", null) + + if (driver != null) Class.forName(driver) + + if ( partitionColumn != null + && (lowerBound == null || upperBound == null || numPartitions == null)) { + sys.error("Partitioning incompletely specified") + } + + val partitionInfo = if (partitionColumn == null) { + null + } else { + 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 { + + 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) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JavaJDBCTrampoline.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JavaJDBCTrampoline.scala new file mode 100644 index 0000000000000..86bb67ec74256 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JavaJDBCTrampoline.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.jdbc + +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); + } +} 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 new file mode 100644 index 0000000000000..34a83f0a5dad8 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala @@ -0,0 +1,235 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.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.sql.types._ + +package object jdbc { + object JDBCWriteDetails extends Logging { + /** + * Returns a PreparedStatement that inserts a row into table via conn. + */ + private def insertStatement(conn: Connection, table: String, rddSchema: StructType): + PreparedStatement = { + val sql = new StringBuilder(s"INSERT INTO $table VALUES (") + var fieldsLeft = rddSchema.fields.length + while (fieldsLeft > 0) { + sql.append("?") + if (fieldsLeft > 1) sql.append(", ") else sql.append(")") + fieldsLeft = fieldsLeft - 1 + } + conn.prepareStatement(sql.toString) + } + + /** + * Saves a partition of a DataFrame to the JDBC database. This is done in + * a single database transaction in order to avoid repeatedly inserting + * data as much as possible. + * + * It is still theoretically possible for rows in a DataFrame to be + * inserted into the database more than once if a stage somehow fails after + * the commit occurs but before the stage can return successfully. + * + * This is not a closure inside saveTable() because apparently cosmetic + * implementation changes elsewhere might easily render such a closure + * non-Serializable. Instead, we explicitly close over all variables that + * are used. + */ + private[jdbc] def savePartition(url: String, table: String, iterator: Iterator[Row], + rddSchema: StructType, nullTypes: Array[Int]): Iterator[Byte] = { + val conn = DriverManager.getConnection(url) + var committed = false + try { + conn.setAutoCommit(false) // Everything in the same db transaction. + val stmt = insertStatement(conn, table, rddSchema) + try { + while (iterator.hasNext) { + val row = iterator.next() + val numFields = rddSchema.fields.length + var i = 0 + while (i < numFields) { + if (row.isNullAt(i)) { + stmt.setNull(i + 1, nullTypes(i)) + } else { + rddSchema.fields(i).dataType match { + case IntegerType => stmt.setInt(i + 1, row.getInt(i)) + case LongType => stmt.setLong(i + 1, row.getLong(i)) + case DoubleType => stmt.setDouble(i + 1, row.getDouble(i)) + case FloatType => stmt.setFloat(i + 1, row.getFloat(i)) + case ShortType => stmt.setInt(i + 1, row.getShort(i)) + case ByteType => stmt.setInt(i + 1, row.getByte(i)) + case BooleanType => stmt.setBoolean(i + 1, row.getBoolean(i)) + case StringType => stmt.setString(i + 1, row.getString(i)) + case BinaryType => stmt.setBytes(i + 1, row.getAs[Array[Byte]](i)) + case TimestampType => stmt.setTimestamp(i + 1, row.getAs[java.sql.Timestamp](i)) + case DateType => stmt.setDate(i + 1, row.getAs[java.sql.Date](i)) + case DecimalType.Unlimited => stmt.setBigDecimal(i + 1, + row.getAs[java.math.BigDecimal](i)) + case _ => throw new IllegalArgumentException( + s"Can't translate non-null value for field $i") + } + } + i = i + 1 + } + stmt.executeUpdate() + } + } finally { + stmt.close() + } + conn.commit() + committed = true + } finally { + if (!committed) { + // The stage must fail. We got here through an exception path, so + // let the exception through unless rollback() or close() want to + // tell the user about another problem. + conn.rollback() + conn.close() + } else { + // The stage must succeed. We cannot propagate any exception close() might throw. + try { + conn.close() + } catch { + case e: Exception => logWarning("Transaction succeeded, but closing failed", e) + } + } + } + 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 = { + val sb = new StringBuilder() + val quirks = DriverQuirks.get(url) + rdd.schema.fields foreach { field => { + val name = field.name + var typ: String = quirks.getJDBCType(field.dataType)._1 + if (typ == null) typ = field.dataType match { + case IntegerType => "INTEGER" + case LongType => "BIGINT" + case DoubleType => "DOUBLE PRECISION" + case FloatType => "REAL" + case ShortType => "INTEGER" + case ByteType => "BYTE" + case BooleanType => "BIT(1)" + case StringType => "TEXT" + case BinaryType => "BLOB" + case TimestampType => "TIMESTAMP" + case DateType => "DATE" + case DecimalType.Unlimited => "DECIMAL(40,20)" + case _ => throw new IllegalArgumentException(s"Don't know how to save $field to JDBC") + } + val nullable = if (field.nullable) "" else "NOT NULL" + sb.append(s", $name $typ $nullable") + }} + if (sb.length < 2) "" else sb.substring(2) + } + + /** + * Saves the RDD to the database in a single transaction. + */ + private def saveTable(url: String, table: String) { + val quirks = DriverQuirks.get(url) + var nullTypes: Array[Int] = rdd.schema.fields.map(field => { + var nullType: Option[Int] = quirks.getJDBCType(field.dataType)._2 + if (nullType.isEmpty) { + field.dataType match { + case IntegerType => java.sql.Types.INTEGER + case LongType => java.sql.Types.BIGINT + case DoubleType => java.sql.Types.DOUBLE + case FloatType => java.sql.Types.REAL + case ShortType => java.sql.Types.INTEGER + case ByteType => java.sql.Types.INTEGER + case BooleanType => java.sql.Types.BIT + case StringType => java.sql.Types.CLOB + case BinaryType => java.sql.Types.BLOB + case TimestampType => java.sql.Types.TIMESTAMP + 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") + } + } 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() + } + 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/test/java/org/apache/spark/sql/jdbc/JavaJDBCTest.java b/sql/core/src/test/java/org/apache/spark/sql/jdbc/JavaJDBCTest.java new file mode 100644 index 0000000000000..80bd74f5b5525 --- /dev/null +++ b/sql/core/src/test/java/org/apache/spark/sql/jdbc/JavaJDBCTest.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.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/DockerHacks.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/DockerHacks.scala new file mode 100644 index 0000000000000..f332cb389f339 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/DockerHacks.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.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/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala new file mode 100644 index 0000000000000..d25c1390db15c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -0,0 +1,248 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 org.apache.spark.sql.test._ +import org.scalatest.{FunSuite, BeforeAndAfter} +import java.sql.DriverManager +import TestSQLContext._ + +class JDBCSuite extends FunSuite with BeforeAndAfter { + val url = "jdbc:h2:mem:testdb0" + 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) + 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() + + sql( + s""" + |CREATE TEMPORARY TABLE foobar + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.PEOPLE') + """.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') + """.stripMargin.replaceAll("\n", " ")) + + conn.prepareStatement("create table test.inttypes (a INT, b BOOLEAN, c TINYINT, " + + "d SMALLINT, e BIGINT)").executeUpdate() + conn.prepareStatement("insert into test.inttypes values (1, false, 3, 4, 1234567890123)" + ).executeUpdate() + conn.prepareStatement("insert into test.inttypes values (null, null, null, null, null)" + ).executeUpdate() + conn.commit() + sql( + s""" + |CREATE TEMPORARY TABLE inttypes + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.INTTYPES') + """.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 (?, ?, ?, ?, ?, ?)") + stmt.setBytes(1, testBytes) + stmt.setString(2, "Sensitive") + stmt.setString(3, "Insensitive") + stmt.setString(4, "Twenty-byte CHAR") + stmt.setBytes(5, testBytes) + stmt.setString(6, "I am a clob!") + stmt.executeUpdate() + sql( + s""" + |CREATE TEMPORARY TABLE strtypes + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.STRTYPES') + """.stripMargin.replaceAll("\n", " ")) + + conn.prepareStatement("create table test.timetypes (a TIME, b DATE, c TIMESTAMP)" + ).executeUpdate() + conn.prepareStatement("insert into test.timetypes values ('12:34:56', " + + "'1996-01-01', '2002-02-20 11:22:33.543543543')").executeUpdate() + conn.commit() + sql( + s""" + |CREATE TEMPORARY TABLE timetypes + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.TIMETYPES') + """.stripMargin.replaceAll("\n", " ")) + + + conn.prepareStatement("create table test.flttypes (a DOUBLE, b REAL, c DECIMAL(40, 20))" + ).executeUpdate() + conn.prepareStatement("insert into test.flttypes values (" + + "1.0000000000000002220446049250313080847263336181640625, " + + "1.00000011920928955078125, " + + "123456789012345.543215432154321)").executeUpdate() + conn.commit() + sql( + s""" + |CREATE TEMPORARY TABLE flttypes + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.FLTTYPES') + """.stripMargin.replaceAll("\n", " ")) + + // Untested: IDENTITY, OTHER, UUID, ARRAY, and GEOMETRY types. + } + + after { + conn.close() + } + + test("SELECT *") { + 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) + } + + 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(2).equals("mary")) + } + + 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) + } + + test("SELECT * partitioned") { + assert(sql("SELECT * FROM parts").collect().size == 3) + } + + 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) + } + + 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) + } + + test("Basic API") { + assert(TestSQLContext.jdbcRDD(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) + } + + 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) + } + + 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) + } + + test("H2 null entries") { + val rows = sql("SELECT * FROM inttypes WHERE A IS NULL").collect() + assert(rows.size == 1) + assert(rows(0).isNullAt(0)) + assert(rows(0).isNullAt(1)) + assert(rows(0).isNullAt(2)) + assert(rows(0).isNullAt(3)) + assert(rows(0).isNullAt(4)) + } + + test("H2 string types") { + val rows = sql("SELECT * FROM strtypes").collect() + assert(rows(0).getAs[Array[Byte]](0).sameElements(testBytes)) + assert(rows(0).getString(1).equals("Sensitive")) + assert(rows(0).getString(2).equals("Insensitive")) + assert(rows(0).getString(3).equals("Twenty-byte CHAR")) + assert(rows(0).getAs[Array[Byte]](4).sameElements(testBytes)) + 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) + 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).getAs[BigDecimal](2) + .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)') + """.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) + } +} 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 new file mode 100644 index 0000000000000..e581ac9b50c2b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -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. + */ + +package org.apache.spark.sql.jdbc + +import java.math.BigDecimal +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._ + +class JDBCWriteSuite extends FunSuite with BeforeAndAfter { + val url = "jdbc:h2:mem:testdb2" + var conn: java.sql.Connection = null + + before { + Class.forName("org.h2.Driver") + conn = DriverManager.getConnection(url) + conn.prepareStatement("create schema test").executeUpdate() + } + + after { + conn.close() + } + + val sc = TestSQLContext.sparkContext + + val arr2x2 = Array[Row](Row.apply("dave", 42), Row.apply("mary", 222)) + val arr1x2 = Array[Row](Row.apply("fred", 3)) + val schema2 = StructType( + StructField("name", StringType) :: + StructField("id", IntegerType) :: Nil) + + val arr2x3 = Array[Row](Row.apply("dave", 42, 1), Row.apply("mary", 222, 2)) + val schema3 = StructType( + StructField("name", StringType) :: + StructField("id", IntegerType) :: + StructField("seq", IntegerType) :: Nil) + + test("Basic CREATE") { + val srdd = TestSQLContext.applySchema(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) + } + + test("CREATE with overwrite") { + val srdd = TestSQLContext.applySchema(sc.parallelize(arr2x3), schema3) + val srdd2 = TestSQLContext.applySchema(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) + + srdd2.createJDBCTable(url, "TEST.DROPTEST", true) + assert(1 == TestSQLContext.jdbcRDD(url, "TEST.DROPTEST").count) + assert(2 == TestSQLContext.jdbcRDD(url, "TEST.DROPTEST").collect()(0).length) + } + + test("CREATE then INSERT to append") { + val srdd = TestSQLContext.applySchema(sc.parallelize(arr2x2), schema2) + val srdd2 = TestSQLContext.applySchema(sc.parallelize(arr1x2), schema2) + + 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) + } + + test("CREATE then INSERT to truncate") { + val srdd = TestSQLContext.applySchema(sc.parallelize(arr2x2), schema2) + val srdd2 = TestSQLContext.applySchema(sc.parallelize(arr1x2), schema2) + + 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) + } + + test("Incompatible INSERT to append") { + val srdd = TestSQLContext.applySchema(sc.parallelize(arr2x2), schema2) + val srdd2 = TestSQLContext.applySchema(sc.parallelize(arr2x3), schema3) + + srdd.createJDBCTable(url, "TEST.INCOMPATIBLETEST", false) + intercept[org.apache.spark.SparkException] { + srdd2.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 new file mode 100644 index 0000000000000..89920f2650c3a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegration.scala @@ -0,0 +1,235 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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, DriverManager, Timestamp} +import com.spotify.docker.client.{DefaultDockerClient, DockerClient} +import com.spotify.docker.client.messages.ContainerConfig +import org.scalatest.{FunSuite, BeforeAndAfterAll, 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() + 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 rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "tbl") + val rows = rdd.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 rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "numbers") + val rows = rdd.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 rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "dates") + val rows = rdd.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 rdd = TestSQLContext.jdbcRDD(url(ip, "foo"), "strings") + val rows = rdd.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 rdd1 = TestSQLContext.jdbcRDD(url(ip, "foo"), "numbers") + val rdd2 = TestSQLContext.jdbcRDD(url(ip, "foo"), "dates") + val rdd3 = TestSQLContext.jdbcRDD(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 new file mode 100644 index 0000000000000..c174d7adb7204 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegration.scala @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.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.messages.ContainerConfig + +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 rdd = TestSQLContext.jdbcRDD(url(db.ip), "public.bar") + val rows = rdd.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 rdd = TestSQLContext.jdbcRDD(url(db.ip), "public.bar") + rdd.createJDBCTable(url(db.ip), "public.barcopy", false) + // Test only that it doesn't bomb out. + } +} From cb39f120430fe44695f38d80858c9aed79eedd6f Mon Sep 17 00:00:00 2001 From: nemccarthy Date: Mon, 2 Feb 2015 20:03:13 -0800 Subject: [PATCH 027/578] [SPARK-5543][WebUI] Remove unused import JsonUtil from from JsonProtocol Simple PR to Remove unused import JsonUtil from from org.apache.spark.util.JsonProtocol.scala which fails builds with older versions of hadoop-core This import is unused. It was introduced in PR #4029 https://github.com/apache/spark/pull/4029 as a part of JIRA SPARK-5231 Author: nemccarthy Closes #4320 from nemccarthy/master and squashes the following commits: 8e34a11 [nemccarthy] [SPARK-5543][WebUI] Remove unused import JsonUtil from from org.apache.spark.util.JsonProtocol.scala which fails builds with older versions of hadoop-core --- core/src/main/scala/org/apache/spark/util/JsonProtocol.scala | 1 - 1 file changed, 1 deletion(-) 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 414bc49a57f8a..8e0e41ad3782e 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -32,7 +32,6 @@ import org.apache.spark.executor._ import org.apache.spark.scheduler._ import org.apache.spark.storage._ import org.apache.spark._ -import org.apache.hadoop.hdfs.web.JsonUtil /** * Serializes SparkListener events to/from JSON. This protocol provides strong backwards- From 0ef38f5fad637fe96464778c1e5ec99cfcc6313c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 2 Feb 2015 21:00:30 -0800 Subject: [PATCH 028/578] SPARK-5542: Decouple publishing, packaging, and tagging in release script These are some changes to the build script to allow parts of it to be run independently. This has already been tested during the 1.2.1 release cycle. Author: Patrick Wendell Author: Patrick Wendell Closes #4319 from pwendell/release-updates and squashes the following commits: dfe7ed9 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into release-updates 478b072 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into release-updates 126dd0c [Patrick Wendell] Allow decoupling Maven publishing from cutting release --- dev/create-release/create-release.sh | 188 ++++++++++++++------------- 1 file changed, 99 insertions(+), 89 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index b2a7e092a0291..607ce1c803507 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -22,8 +22,9 @@ # Expects to be run in a totally empty directory. # # Options: -# --package-only only packages an existing release candidate -# +# --skip-create-release Assume the desired release tag already exists +# --skip-publish Do not publish to Maven central +# --skip-package Do not package and upload binary artifacts # Would be nice to add: # - Send output to stderr and have useful logging in stdout @@ -51,7 +52,7 @@ set -e GIT_TAG=v$RELEASE_VERSION-$RC_NAME -if [[ ! "$@" =~ --package-only ]]; then +if [[ ! "$@" =~ --skip-create-release ]]; then echo "Creating release commit and publishing to Apache repository" # Artifact publishing git clone https://$ASF_USERNAME:$ASF_PASSWORD@git-wip-us.apache.org/repos/asf/spark.git \ @@ -87,8 +88,15 @@ if [[ ! "$@" =~ --package-only ]]; then git commit -a -m "Preparing development version $next_ver" git push origin $GIT_TAG git push origin HEAD:$GIT_BRANCH - git checkout -f $GIT_TAG + popd + rm -rf spark +fi +if [[ ! "$@" =~ --skip-publish ]]; then + git clone https://$ASF_USERNAME:$ASF_PASSWORD@git-wip-us.apache.org/repos/asf/spark.git + pushd spark + git checkout --force $GIT_TAG + # Using Nexus API documented here: # https://support.sonatype.com/entries/39720203-Uploading-to-a-Staging-Repository-via-REST-API echo "Creating Nexus staging repository" @@ -106,7 +114,7 @@ if [[ ! "$@" =~ --package-only ]]; then clean install ./dev/change-version-to-2.11.sh - + 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 @@ -155,88 +163,90 @@ if [[ ! "$@" =~ --package-only ]]; then rm -rf spark fi -# Source and binary tarballs -echo "Packaging release tarballs" -git clone https://git-wip-us.apache.org/repos/asf/spark.git -cd spark -git checkout --force $GIT_TAG -release_hash=`git rev-parse HEAD` - -rm .gitignore -rm -rf .git -cd .. - -cp -r spark spark-$RELEASE_VERSION -tar cvzf spark-$RELEASE_VERSION.tgz spark-$RELEASE_VERSION -echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour --output spark-$RELEASE_VERSION.tgz.asc \ - --detach-sig spark-$RELEASE_VERSION.tgz -echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md MD5 spark-$RELEASE_VERSION.tgz > \ - spark-$RELEASE_VERSION.tgz.md5 -echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md SHA512 spark-$RELEASE_VERSION.tgz > \ - spark-$RELEASE_VERSION.tgz.sha -rm -rf spark-$RELEASE_VERSION - -make_binary_release() { - NAME=$1 - FLAGS=$2 - cp -r spark spark-$RELEASE_VERSION-bin-$NAME - - cd spark-$RELEASE_VERSION-bin-$NAME - - # TODO There should probably be a flag to make-distribution to allow 2.11 support - if [[ $FLAGS == *scala-2.11* ]]; then - ./dev/change-version-to-2.11.sh - fi - - ./make-distribution.sh --name $NAME --tgz $FLAGS 2>&1 | tee ../binary-release-$NAME.log +if [[ ! "$@" =~ --skip-package ]]; then + # Source and binary tarballs + echo "Packaging release tarballs" + git clone https://git-wip-us.apache.org/repos/asf/spark.git + cd spark + git checkout --force $GIT_TAG + release_hash=`git rev-parse HEAD` + + rm .gitignore + rm -rf .git cd .. - cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz . - rm -rf spark-$RELEASE_VERSION-bin-$NAME - - echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour \ - --output spark-$RELEASE_VERSION-bin-$NAME.tgz.asc \ - --detach-sig spark-$RELEASE_VERSION-bin-$NAME.tgz - echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \ - MD5 spark-$RELEASE_VERSION-bin-$NAME.tgz > \ - spark-$RELEASE_VERSION-bin-$NAME.tgz.md5 - echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \ - SHA512 spark-$RELEASE_VERSION-bin-$NAME.tgz > \ - spark-$RELEASE_VERSION-bin-$NAME.tgz.sha -} - - -make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" & -make_binary_release "hadoop1-scala2.11" "-Phive -Dscala-2.11" & -make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & -make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Phive-thriftserver -Pyarn" & -make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn" & -make_binary_release "mapr3" "-Pmapr3 -Phive -Phive-thriftserver" & -make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive -Phive-thriftserver" & -make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" & -wait - -# Copy data -echo "Copying release tarballs" -rc_folder=spark-$RELEASE_VERSION-$RC_NAME -ssh $ASF_USERNAME@people.apache.org \ - mkdir /home/$ASF_USERNAME/public_html/$rc_folder -scp spark-* \ - $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_folder/ - -# Docs -cd spark -build/sbt clean -cd docs -# Compile docs with Java 7 to use nicer format -JAVA_HOME=$JAVA_7_HOME PRODUCTION=1 jekyll build -echo "Copying release documentation" -rc_docs_folder=${rc_folder}-docs -ssh $ASF_USERNAME@people.apache.org \ - mkdir /home/$ASF_USERNAME/public_html/$rc_docs_folder -rsync -r _site/* $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_docs_folder - -echo "Release $RELEASE_VERSION completed:" -echo "Git tag:\t $GIT_TAG" -echo "Release commit:\t $release_hash" -echo "Binary location:\t http://people.apache.org/~$ASF_USERNAME/$rc_folder" -echo "Doc location:\t http://people.apache.org/~$ASF_USERNAME/$rc_docs_folder" + + cp -r spark spark-$RELEASE_VERSION + tar cvzf spark-$RELEASE_VERSION.tgz spark-$RELEASE_VERSION + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour --output spark-$RELEASE_VERSION.tgz.asc \ + --detach-sig spark-$RELEASE_VERSION.tgz + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md MD5 spark-$RELEASE_VERSION.tgz > \ + spark-$RELEASE_VERSION.tgz.md5 + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md SHA512 spark-$RELEASE_VERSION.tgz > \ + spark-$RELEASE_VERSION.tgz.sha + rm -rf spark-$RELEASE_VERSION + + make_binary_release() { + NAME=$1 + FLAGS=$2 + cp -r spark spark-$RELEASE_VERSION-bin-$NAME + + cd spark-$RELEASE_VERSION-bin-$NAME + + # TODO There should probably be a flag to make-distribution to allow 2.11 support + if [[ $FLAGS == *scala-2.11* ]]; then + ./dev/change-version-to-2.11.sh + fi + + ./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 . + rm -rf spark-$RELEASE_VERSION-bin-$NAME + + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour \ + --output spark-$RELEASE_VERSION-bin-$NAME.tgz.asc \ + --detach-sig spark-$RELEASE_VERSION-bin-$NAME.tgz + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \ + MD5 spark-$RELEASE_VERSION-bin-$NAME.tgz > \ + spark-$RELEASE_VERSION-bin-$NAME.tgz.md5 + echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \ + SHA512 spark-$RELEASE_VERSION-bin-$NAME.tgz > \ + spark-$RELEASE_VERSION-bin-$NAME.tgz.sha + } + + + make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" & + make_binary_release "hadoop1-scala2.11" "-Phive -Dscala-2.11" & + make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & + make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Phive-thriftserver -Pyarn" & + make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn" & + make_binary_release "mapr3" "-Pmapr3 -Phive -Phive-thriftserver" & + make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive -Phive-thriftserver" & + make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" & + wait + + # Copy data + echo "Copying release tarballs" + rc_folder=spark-$RELEASE_VERSION-$RC_NAME + ssh $ASF_USERNAME@people.apache.org \ + mkdir /home/$ASF_USERNAME/public_html/$rc_folder + scp spark-* \ + $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_folder/ + + # Docs + cd spark + sbt/sbt clean + cd docs + # Compile docs with Java 7 to use nicer format + JAVA_HOME=$JAVA_7_HOME PRODUCTION=1 jekyll build + echo "Copying release documentation" + rc_docs_folder=${rc_folder}-docs + ssh $ASF_USERNAME@people.apache.org \ + mkdir /home/$ASF_USERNAME/public_html/$rc_docs_folder + rsync -r _site/* $ASF_USERNAME@people.apache.org:/home/$ASF_USERNAME/public_html/$rc_docs_folder + + echo "Release $RELEASE_VERSION completed:" + echo "Git tag:\t $GIT_TAG" + echo "Release commit:\t $release_hash" + echo "Binary location:\t http://people.apache.org/~$ASF_USERNAME/$rc_folder" + echo "Doc location:\t http://people.apache.org/~$ASF_USERNAME/$rc_docs_folder" +fi From 7930d2bef0e2c7f62456e013124455061dfe6dc8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 2 Feb 2015 21:01:36 -0800 Subject: [PATCH 029/578] SPARK-3996: Add jetty servlet and continuations. These are needed transitively from the other Jetty libraries we include. It was not picked up by unit tests because we disable the UI. Author: Patrick Wendell Closes #4323 from pwendell/jetty and squashes the following commits: d8669da [Patrick Wendell] SPARK-3996: Add jetty servlet and continuations. --- core/pom.xml | 12 +++++++++++- pom.xml | 14 ++++++++++++++ streaming/pom.xml | 4 ++++ yarn/pom.xml | 4 ++++ 4 files changed, 33 insertions(+), 1 deletion(-) diff --git a/core/pom.xml b/core/pom.xml index 4d24ae93c4ae3..2c115683fce66 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -122,6 +122,16 @@ jetty-http compile + + org.eclipse.jetty + jetty-continuation + compile + + + org.eclipse.jetty + jetty-servlet + compile + org.apache.commons @@ -377,7 +387,7 @@ true true - guava,jetty-io,jetty-http,jetty-plus,jetty-util,jetty-server + guava,jetty-io,jetty-servlet,jetty-continuation,jetty-http,jetty-plus,jetty-util,jetty-server true diff --git a/pom.xml b/pom.xml index 08d1cc33e4701..e25eced877578 100644 --- a/pom.xml +++ b/pom.xml @@ -347,6 +347,18 @@ ${jetty.version} provided + + org.eclipse.jetty + jetty-continuation + ${jetty.version} + provided + + + org.eclipse.jetty + jetty-servlet + ${jetty.version} + provided + org.eclipse.jetty jetty-util @@ -1297,6 +1309,8 @@ org.eclipse.jetty:jetty-io org.eclipse.jetty:jetty-http + org.eclipse.jetty:jetty-continuation + org.eclipse.jetty:jetty-servlet org.eclipse.jetty:jetty-plus org.eclipse.jetty:jetty-security org.eclipse.jetty:jetty-util diff --git a/streaming/pom.xml b/streaming/pom.xml index 5efed16039339..1e92ba686a57d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -62,6 +62,10 @@ org.eclipse.jetty jetty-http + + org.eclipse.jetty + jetty-servlet + diff --git a/yarn/pom.xml b/yarn/pom.xml index 6bdf9d2416823..65344aa8738e0 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -80,6 +80,10 @@ org.eclipse.jetty jetty-http + + org.eclipse.jetty + jetty-servlet + From 60f67e7a142f831f91f60676f94affa8add9944f Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 2 Feb 2015 21:14:21 -0800 Subject: [PATCH 030/578] [Doc] Minor: Fixes several formatting issues Fixes several minor formatting issues in the [Continuous Compilation] [1] section. [1]: http://spark.apache.org/docs/latest/building-spark.html#continuous-compilation [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4316) Author: Cheng Lian Closes #4316 from liancheng/fix-build-instruction-docs and squashes the following commits: 0a92e01 [Cheng Lian] Fixes several formatting issues --- docs/building-spark.md | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index fb93017861ed0..db69905813817 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -137,15 +137,18 @@ We use the scala-maven-plugin which supports incremental and continuous compilat should run continuous compilation (i.e. wait for changes). However, this has not been tested extensively. A couple of gotchas to note: + * it only scans the paths `src/main` and `src/test` (see [docs](http://scala-tools.org/mvnsites/maven-scala-plugin/usage_cc.html)), so it will only work from within certain submodules that have that structure. + * you'll typically need to run `mvn install` from the project root for compilation within specific submodules to work; this is because submodules that depend on other submodules do so via the `spark-parent` module). Thus, the full flow for running continuous-compilation of the `core` submodule may look more like: - ``` + +``` $ mvn install $ cd core $ mvn scala:cc From c306555f491e45ef870f58938af397f9ec5f166a Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 2 Feb 2015 21:42:18 -0800 Subject: [PATCH 031/578] [SPARK-5219][Core] Add locks to avoid scheduling race conditions Author: zsxwing Closes #4019 from zsxwing/SPARK-5219 and squashes the following commits: 36a8b4e [zsxwing] Add locks to avoid race conditions --- .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 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 33a7aae5d3fcd..79f84e70df9d5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -361,7 +361,7 @@ private[spark] class TaskSchedulerImpl( dagScheduler.executorHeartbeatReceived(execId, metricsWithStageIds, blockManagerId) } - def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long) { + def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long): Unit = synchronized { taskSetManager.handleTaskGettingResult(tid) } 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 5c94c6bbcb37b..97c22fe724abd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -542,7 +542,7 @@ private[spark] class TaskSetManager( /** * Check whether has enough quota to fetch the result with `size` bytes */ - def canFetchMoreResults(size: Long): Boolean = synchronized { + def canFetchMoreResults(size: Long): Boolean = sched.synchronized { totalResultSize += size calculatedTasks += 1 if (maxResultSize > 0 && totalResultSize > maxResultSize) { @@ -671,7 +671,7 @@ private[spark] class TaskSetManager( maybeFinishTaskSet() } - def abort(message: String) { + def abort(message: String): Unit = sched.synchronized { // TODO: Kill running tasks if we were not terminated due to a Mesos error sched.dagScheduler.taskSetFailed(taskSet, message) isZombie = true From eb0da6c4bd55aaab972c53eb934e68257b8994e5 Mon Sep 17 00:00:00 2001 From: freeman Date: Mon, 2 Feb 2015 22:42:15 -0800 Subject: [PATCH 032/578] [SPARK-4979][MLLIB] Streaming logisitic regression This adds support for streaming logistic regression with stochastic gradient descent, in the same manner as the existing implementation of streaming linear regression. It is a relatively simple addition because most of the work is already done by the abstract class `StreamingLinearAlgorithm` and existing algorithms and models from MLlib. The PR includes - Streaming Logistic Regression algorithm - Unit tests for accuracy, streaming convergence, and streaming prediction - An example use cc mengxr tdas Author: freeman Closes #4306 from freeman-lab/streaming-logisitic-regression and squashes the following commits: 5c2c70b [freeman] Use Option on model 5cca2bc [freeman] Merge remote-tracking branch 'upstream/master' into streaming-logisitic-regression 275f8bd [freeman] Make private to mllib 3926e4e [freeman] Line formatting 5ee8694 [freeman] Experimental tag for docs 2fc68ac [freeman] Fix example formatting 85320b1 [freeman] Fixed line length d88f717 [freeman] Remove stray comment 59d7ecb [freeman] Add streaming logistic regression e78fe28 [freeman] Add streaming logistic regression example 321cc66 [freeman] Set private and protected within mllib --- .../mllib/StreamingLinearRegression.scala | 3 +- .../mllib/StreamingLogisticRegression.scala | 73 ++++++++++ .../classification/LogisticRegression.scala | 4 +- .../StreamingLogisticRegressionWithSGD.scala | 95 ++++++++++++ .../regression/StreamingLinearAlgorithm.scala | 28 ++-- .../StreamingLinearRegressionWithSGD.scala | 16 +-- .../StreamingLogisticRegressionSuite.scala | 135 ++++++++++++++++++ 7 files changed, 327 insertions(+), 27 deletions(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionWithSGD.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala index c5bd5b0b178d9..1a95048bbfe2d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala @@ -35,8 +35,7 @@ import org.apache.spark.streaming.{Seconds, StreamingContext} * * To run on your local machine using the two directories `trainingDir` and `testDir`, * with updates every 5 seconds, and 2 features per data point, call: - * $ bin/run-example \ - * org.apache.spark.examples.mllib.StreamingLinearRegression trainingDir testDir 5 2 + * $ bin/run-example mllib.StreamingLinearRegression trainingDir testDir 5 2 * * As you add text files to `trainingDir` the model will continuously update. * Anytime you add text files to `testDir`, you'll see predictions from the current model. diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala new file mode 100644 index 0000000000000..e1998099c2d78 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.classification.StreamingLogisticRegressionWithSGD +import org.apache.spark.SparkConf +import org.apache.spark.streaming.{Seconds, StreamingContext} + +/** + * Train a logistic regression model on one stream of data and make predictions + * on another stream, where the data streams arrive as text files + * into two different directories. + * + * The rows of the text files must be labeled data points in the form + * `(y,[x1,x2,x3,...,xn])` + * Where n is the number of features, y is a binary label, and + * n must be the same for train and test. + * + * Usage: StreamingLogisticRegression + * + * To run on your local machine using the two directories `trainingDir` and `testDir`, + * with updates every 5 seconds, and 2 features per data point, call: + * $ bin/run-example mllib.StreamingLogisticRegression trainingDir testDir 5 2 + * + * As you add text files to `trainingDir` the model will continuously update. + * Anytime you add text files to `testDir`, you'll see predictions from the current model. + * + */ +object StreamingLogisticRegression { + + def main(args: Array[String]) { + + if (args.length != 4) { + System.err.println( + "Usage: StreamingLogisticRegression ") + System.exit(1) + } + + val conf = new SparkConf().setMaster("local").setAppName("StreamingLogisticRegression") + val ssc = new StreamingContext(conf, Seconds(args(2).toLong)) + + val trainingData = ssc.textFileStream(args(0)).map(LabeledPoint.parse) + val testData = ssc.textFileStream(args(1)).map(LabeledPoint.parse) + + val model = new StreamingLogisticRegressionWithSGD() + .setInitialWeights(Vectors.zeros(args(3).toInt)) + + model.trainOn(trainingData) + model.predictOnValues(testData.map(lp => (lp.label, lp.features))).print() + + ssc.start() + ssc.awaitTermination() + + } + +} 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 282fb3ff283f4..a469315a1b5c3 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 @@ -136,7 +136,7 @@ class LogisticRegressionModel ( * for k classes multi-label classification problem. * Using [[LogisticRegressionWithLBFGS]] is recommended over this. */ -class LogisticRegressionWithSGD private ( +class LogisticRegressionWithSGD private[mllib] ( private var stepSize: Double, private var numIterations: Int, private var regParam: Double, @@ -158,7 +158,7 @@ class LogisticRegressionWithSGD private ( */ def this() = this(1.0, 100, 0.01, 1.0) - override protected def createModel(weights: Vector, intercept: Double) = { + override protected[mllib] def createModel(weights: Vector, intercept: Double) = { new LogisticRegressionModel(weights, intercept) } } 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 new file mode 100644 index 0000000000000..eabd2162e287f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionWithSGD.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.mllib.classification + +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.StreamingLinearAlgorithm + +/** + * :: Experimental :: + * Train or predict a logistic regression model on streaming data. Training uses + * Stochastic Gradient Descent to update the model based on each new batch of + * incoming data from a DStream (see `LogisticRegressionWithSGD` for model equation) + * + * Each batch of data is assumed to be an RDD of LabeledPoints. + * The number of data points per batch can vary, but the number + * of features must be constant. An initial weight + * vector must be provided. + * + * Use a builder pattern to construct a streaming logistic regression + * analysis in an application, like: + * + * val model = new StreamingLogisticRegressionWithSGD() + * .setStepSize(0.5) + * .setNumIterations(10) + * .setInitialWeights(Vectors.dense(...)) + * .trainOn(DStream) + * + */ +@Experimental +class StreamingLogisticRegressionWithSGD private[mllib] ( + private var stepSize: Double, + private var numIterations: Int, + private var miniBatchFraction: Double, + private var regParam: Double) + extends StreamingLinearAlgorithm[LogisticRegressionModel, LogisticRegressionWithSGD] + with Serializable { + + /** + * Construct a StreamingLogisticRegression object with default parameters: + * {stepSize: 0.1, numIterations: 50, miniBatchFraction: 1.0, regParam: 0.0}. + * Initial weights must be set before using trainOn or predictOn + * (see `StreamingLinearAlgorithm`) + */ + def this() = this(0.1, 50, 1.0, 0.0) + + val algorithm = new LogisticRegressionWithSGD( + stepSize, numIterations, regParam, miniBatchFraction) + + /** Set the step size for gradient descent. Default: 0.1. */ + def setStepSize(stepSize: Double): this.type = { + this.algorithm.optimizer.setStepSize(stepSize) + this + } + + /** Set the number of iterations of gradient descent to run per update. Default: 50. */ + def setNumIterations(numIterations: Int): this.type = { + this.algorithm.optimizer.setNumIterations(numIterations) + this + } + + /** Set the fraction of each batch to use for updates. Default: 1.0. */ + def setMiniBatchFraction(miniBatchFraction: Double): this.type = { + this.algorithm.optimizer.setMiniBatchFraction(miniBatchFraction) + this + } + + /** Set the regularization parameter. Default: 0.0. */ + def setRegParam(regParam: Double): this.type = { + this.algorithm.optimizer.setRegParam(regParam) + this + } + + /** Set the initial weights. Default: [0.0, 0.0]. */ + def setInitialWeights(initialWeights: Vector): this.type = { + this.model = Option(algorithm.createModel(initialWeights, 0.0)) + this + } + +} 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 b549b7c475fc3..39a0dee931d3d 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 @@ -58,14 +58,14 @@ abstract class StreamingLinearAlgorithm[ A <: GeneralizedLinearAlgorithm[M]] extends Logging { /** The model to be updated and used for prediction. */ - protected var model: M + protected var model: Option[M] = null /** The algorithm to use for updating. */ protected val algorithm: A /** Return the latest model. */ def latestModel(): M = { - model + model.get } /** @@ -77,16 +77,16 @@ abstract class StreamingLinearAlgorithm[ * @param data DStream containing labeled data */ def trainOn(data: DStream[LabeledPoint]) { - if (Option(model.weights) == None) { - logError("Initial weights must be set before starting training") + if (Option(model) == None) { + logError("Model must be initialized before starting training") throw new IllegalArgumentException } data.foreachRDD { (rdd, time) => - model = algorithm.run(rdd, model.weights) + model = Option(algorithm.run(rdd, model.get.weights)) logInfo("Model updated at time %s".format(time.toString)) - val display = model.weights.size match { - case x if x > 100 => model.weights.toArray.take(100).mkString("[", ",", "...") - case _ => model.weights.toArray.mkString("[", ",", "]") + val display = model.get.weights.size match { + case x if x > 100 => model.get.weights.toArray.take(100).mkString("[", ",", "...") + case _ => model.get.weights.toArray.mkString("[", ",", "]") } logInfo("Current model: weights, %s".format (display)) } @@ -99,12 +99,12 @@ abstract class StreamingLinearAlgorithm[ * @return DStream containing predictions */ def predictOn(data: DStream[Vector]): DStream[Double] = { - if (Option(model.weights) == None) { - val msg = "Initial weights must be set before starting prediction" + if (Option(model) == None) { + val msg = "Model must be initialized before starting prediction" logError(msg) throw new IllegalArgumentException(msg) } - data.map(model.predict) + data.map(model.get.predict) } /** @@ -114,11 +114,11 @@ abstract class StreamingLinearAlgorithm[ * @return DStream containing the input keys and the predictions as values */ def predictOnValues[K: ClassTag](data: DStream[(K, Vector)]): DStream[(K, Double)] = { - if (Option(model.weights) == None) { - val msg = "Initial weights must be set before starting prediction" + if (Option(model) == None) { + val msg = "Model must be initialized before starting prediction" logError(msg) throw new IllegalArgumentException(msg) } - data.mapValues(model.predict) + data.mapValues(model.get.predict) } } 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 1d11fde24712c..c0625b4880953 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 @@ -21,6 +21,7 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.Vector /** + * :: Experimental :: * Train or predict a linear regression model on streaming data. Training uses * Stochastic Gradient Descent to update the model based on each new batch of * incoming data from a DStream (see `LinearRegressionWithSGD` for model equation) @@ -41,13 +42,12 @@ import org.apache.spark.mllib.linalg.Vector * */ @Experimental -class StreamingLinearRegressionWithSGD ( +class StreamingLinearRegressionWithSGD private[mllib] ( private var stepSize: Double, private var numIterations: Int, - private var miniBatchFraction: Double, - private var initialWeights: Vector) - extends StreamingLinearAlgorithm[ - LinearRegressionModel, LinearRegressionWithSGD] with Serializable { + private var miniBatchFraction: Double) + extends StreamingLinearAlgorithm[LinearRegressionModel, LinearRegressionWithSGD] + with Serializable { /** * Construct a StreamingLinearRegression object with default parameters: @@ -55,12 +55,10 @@ class StreamingLinearRegressionWithSGD ( * Initial weights must be set before using trainOn or predictOn * (see `StreamingLinearAlgorithm`) */ - def this() = this(0.1, 50, 1.0, null) + def this() = this(0.1, 50, 1.0) val algorithm = new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction) - var model = algorithm.createModel(initialWeights, 0.0) - /** Set the step size for gradient descent. Default: 0.1. */ def setStepSize(stepSize: Double): this.type = { this.algorithm.optimizer.setStepSize(stepSize) @@ -81,7 +79,7 @@ class StreamingLinearRegressionWithSGD ( /** Set the initial weights. Default: [0.0, 0.0]. */ def setInitialWeights(initialWeights: Vector): this.type = { - this.model = algorithm.createModel(initialWeights, 0.0) + this.model = Option(algorithm.createModel(initialWeights, 0.0)) this } 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 new file mode 100644 index 0000000000000..8b3e6e5ce9249 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.classification + +import scala.collection.mutable.ArrayBuffer + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.TestSuiteBase + +class StreamingLogisticRegressionSuite extends FunSuite with TestSuiteBase { + + // use longer wait time to ensure job completion + override def maxWaitTimeMillis = 30000 + + // Test if we can accurately learn B for Y = logistic(BX) on streaming data + test("parameter accuracy") { + + val nPoints = 100 + val B = 1.5 + + // create model + val model = new StreamingLogisticRegressionWithSGD() + .setInitialWeights(Vectors.dense(0.0)) + .setStepSize(0.2) + .setNumIterations(25) + + // generate sequence of simulated data + val numBatches = 20 + val input = (0 until numBatches).map { i => + LogisticRegressionSuite.generateLogisticInput(0.0, B, nPoints, 42 * (i + 1)) + } + + // apply model training to input stream + val ssc = setupStreams(input, (inputDStream: DStream[LabeledPoint]) => { + model.trainOn(inputDStream) + inputDStream.count() + }) + runStreams(ssc, numBatches, numBatches) + + // check accuracy of final parameter estimates + assert(model.latestModel().weights(0) ~== B relTol 0.1) + + } + + // Test that parameter estimates improve when learning Y = logistic(BX) on streaming data + test("parameter convergence") { + + val B = 1.5 + val nPoints = 100 + + // create model + val model = new StreamingLogisticRegressionWithSGD() + .setInitialWeights(Vectors.dense(0.0)) + .setStepSize(0.2) + .setNumIterations(25) + + // generate sequence of simulated data + val numBatches = 20 + val input = (0 until numBatches).map { i => + LogisticRegressionSuite.generateLogisticInput(0.0, B, nPoints, 42 * (i + 1)) + } + + // create buffer to store intermediate fits + val history = new ArrayBuffer[Double](numBatches) + + // apply model training to input stream, storing the intermediate results + // (we add a count to ensure the result is a DStream) + val ssc = setupStreams(input, (inputDStream: DStream[LabeledPoint]) => { + model.trainOn(inputDStream) + inputDStream.foreachRDD(x => history.append(math.abs(model.latestModel().weights(0) - B))) + inputDStream.count() + }) + runStreams(ssc, numBatches, numBatches) + + // compute change in error + val deltas = history.drop(1).zip(history.dropRight(1)) + // check error stability (it always either shrinks, or increases with small tol) + assert(deltas.forall(x => (x._1 - x._2) <= 0.1)) + // check that error shrunk on at least 2 batches + assert(deltas.map(x => if ((x._1 - x._2) < 0) 1 else 0).sum > 1) + } + + // Test predictions on a stream + test("predictions") { + + val B = 1.5 + val nPoints = 100 + + // create model initialized with true weights + val model = new StreamingLogisticRegressionWithSGD() + .setInitialWeights(Vectors.dense(1.5)) + .setStepSize(0.2) + .setNumIterations(25) + + // generate sequence of simulated data for testing + val numBatches = 10 + val testInput = (0 until numBatches).map { i => + LogisticRegressionSuite.generateLogisticInput(0.0, B, nPoints, 42 * (i + 1)) + } + + // apply model predictions to test stream + val ssc = setupStreams(testInput, (inputDStream: DStream[LabeledPoint]) => { + model.predictOnValues(inputDStream.map(x => (x.label, x.features))) + }) + + // collect the output as (true, estimated) tuples + val output: Seq[Seq[(Double, Double)]] = runStreams(ssc, numBatches, numBatches) + + // check that at least 60% of predictions are correct on all batches + val errors = output.map(batch => batch.map(p => math.abs(p._1 - p._2)).sum / nPoints) + + assert(errors.forall(x => x <= 0.4)) + } + +} From c31c36c4a76bd3449696383321332ec95bff7fed Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Mon, 2 Feb 2015 22:45:55 -0800 Subject: [PATCH 033/578] [SPARK-3778] newAPIHadoopRDD doesn't properly pass credentials for secure hdfs .this was https://github.com/apache/spark/pull/2676 https://issues.apache.org/jira/browse/SPARK-3778 This affects if someone is trying to access secure hdfs something like: val lines = { val hconf = new Configuration() hconf.set("mapred.input.dir", "mydir") hconf.set("textinputformat.record.delimiter","\003432\n") sc.newAPIHadoopRDD(hconf, classOf[TextInputFormat], classOf[LongWritable], classOf[Text]) } Author: Thomas Graves Closes #4292 from tgravescs/SPARK-3788 and squashes the following commits: cf3b453 [Thomas Graves] newAPIHadoopRDD doesn't properly pass credentials for secure hdfs on yarn --- core/src/main/scala/org/apache/spark/SparkContext.scala | 7 ++++++- 1 file changed, 6 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 228076f01c841..6a16a31654630 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -804,6 +804,8 @@ 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, + // so we don't need to explicitly add them ourselves val job = new NewHadoopJob(conf) NewFileInputFormat.addInputPath(job, new Path(path)) val updatedConf = job.getConfiguration @@ -826,7 +828,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli kClass: Class[K], vClass: Class[V]): RDD[(K, V)] = { assertNotStopped() - new NewHadoopRDD(this, fClass, kClass, vClass, conf) + // Add necessary security credentials to the JobConf. Required to access secure HDFS. + val jconf = new JobConf(conf) + SparkHadoopUtil.get.addCredentials(jconf) + new NewHadoopRDD(this, fClass, kClass, vClass, jconf) } /** Get an RDD for a Hadoop SequenceFile with given key and value types. From 50a1a874e1d087a6c79835b1936d0009622a97b1 Mon Sep 17 00:00:00 2001 From: FlytxtRnD Date: Mon, 2 Feb 2015 23:04:55 -0800 Subject: [PATCH 034/578] [SPARK-5012][MLLib][PySpark]Python API for Gaussian Mixture Model Python API for the Gaussian Mixture Model clustering algorithm in MLLib. Author: FlytxtRnD Closes #4059 from FlytxtRnD/PythonGmmWrapper and squashes the following commits: c973ab3 [FlytxtRnD] Merge branch 'PythonGmmWrapper', remote-tracking branch 'upstream/master' into PythonGmmWrapper 339b09c [FlytxtRnD] Added MultivariateGaussian namedtuple and Arraybuffer in trainGaussianMixture fa0a142 [FlytxtRnD] New line added d5b36ab [FlytxtRnD] Changed argument names to lowercase ac134f1 [FlytxtRnD] Merge branch 'PythonGmmWrapper' of https://github.com/FlytxtRnD/spark into PythonGmmWrapper 6671ea1 [FlytxtRnD] Added mllib/stat/distribution.py 3aee84b [FlytxtRnD] Fixed style issues 2e9f12a [FlytxtRnD] Added mllib/stat/distribution.py and fixed style issues b22532c [FlytxtRnD] Merge branch 'PythonGmmWrapper', remote-tracking branch 'upstream/master' into PythonGmmWrapper 2e14d82 [FlytxtRnD] Incorporate MultivariateGaussian instances in GaussianMixtureModel 05767c7 [FlytxtRnD] Merge branch 'PythonGmmWrapper', remote-tracking branch 'upstream/master' into PythonGmmWrapper 3464d19 [FlytxtRnD] Merge branch 'PythonGmmWrapper', remote-tracking branch 'upstream/master' into PythonGmmWrapper c1d4c71 [FlytxtRnD] Merge branch 'PythonGmmWrapper', remote-tracking branch 'origin/PythonGmmWrapper' into PythonGmmWrapper 426d130 [FlytxtRnD] Added random seed parameter 332bad1 [FlytxtRnD] Merge branch 'PythonGmmWrapper', remote-tracking branch 'upstream/master' into PythonGmmWrapper f82750b [FlytxtRnD] Fixed style issues 5c83825 [FlytxtRnD] Split input file with space delimiter fda60f3 [FlytxtRnD] Python API for Gaussian Mixture Model --- .../python/mllib/gaussian_mixture_model.py | 65 +++++++++++++ .../mllib/api/python/PythonMLLibAPI.scala | 56 ++++++++++- python/pyspark/mllib/clustering.py | 92 ++++++++++++++++++- python/pyspark/mllib/stat/__init__.py | 3 +- python/pyspark/mllib/stat/distribution.py | 31 +++++++ python/pyspark/mllib/tests.py | 26 ++++++ 6 files changed, 267 insertions(+), 6 deletions(-) create mode 100644 examples/src/main/python/mllib/gaussian_mixture_model.py create mode 100644 python/pyspark/mllib/stat/distribution.py diff --git a/examples/src/main/python/mllib/gaussian_mixture_model.py b/examples/src/main/python/mllib/gaussian_mixture_model.py new file mode 100644 index 0000000000000..a2cd626c9f19d --- /dev/null +++ b/examples/src/main/python/mllib/gaussian_mixture_model.py @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT 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 Gaussian Mixture Model clustering program using MLlib. +""" +import sys +import random +import argparse +import numpy as np + +from pyspark import SparkConf, SparkContext +from pyspark.mllib.clustering import GaussianMixture + + +def parseVector(line): + return np.array([float(x) for x in line.split(' ')]) + + +if __name__ == "__main__": + """ + Parameters + ---------- + :param inputFile: Input file path which contains data points + :param k: Number of mixture components + :param convergenceTol: Convergence threshold. Default to 1e-3 + :param maxIterations: Number of EM iterations to perform. Default to 100 + :param seed: Random seed + """ + + parser = argparse.ArgumentParser() + parser.add_argument('inputFile', help='Input File') + parser.add_argument('k', type=int, help='Number of clusters') + parser.add_argument('--convergenceTol', default=1e-3, type=float, help='convergence threshold') + parser.add_argument('--maxIterations', default=100, type=int, help='Number of iterations') + parser.add_argument('--seed', default=random.getrandbits(19), + type=long, help='Random seed') + args = parser.parse_args() + + conf = SparkConf().setAppName("GMM") + sc = SparkContext(conf=conf) + + lines = sc.textFile(args.inputFile) + data = lines.map(parseVector) + 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)) + sc.stop() 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 a66d6f0cf29c7..980980593d194 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 @@ -22,6 +22,7 @@ import java.nio.{ByteBuffer, ByteOrder} import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import scala.language.existentials import scala.reflect.ClassTag @@ -40,6 +41,7 @@ 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} @@ -260,7 +262,7 @@ class PythonMLLibAPI extends Serializable { } /** - * Java stub for Python mllib KMeans.train() + * Java stub for Python mllib KMeans.run() */ def trainKMeansModel( data: JavaRDD[Vector], @@ -284,6 +286,58 @@ class PythonMLLibAPI extends Serializable { } } + /** + * Java stub for Python mllib GaussianMixture.run() + * Returns a list containing weights, mean and covariance of each mixture component. + */ + def trainGaussianMixture( + data: JavaRDD[Vector], + k: Int, + convergenceTol: Double, + maxIterations: Int, + seed: Long): JList[Object] = { + val gmmAlg = new GaussianMixture() + .setK(k) + .setConvergenceTol(convergenceTol) + .setMaxIterations(maxIterations) + + if (seed != null) gmmAlg.setSeed(seed) + + try { + val model = gmmAlg.run(data.rdd.persist(StorageLevel.MEMORY_AND_DISK)) + var wt = ArrayBuffer.empty[Double] + var mu = ArrayBuffer.empty[Vector] + var sigma = ArrayBuffer.empty[Matrix] + for (i <- 0 until model.k) { + wt += model.weights(i) + mu += model.gaussians(i).mu + sigma += model.gaussians(i).sigma + } + List(wt.toArray, mu.toArray, sigma.toArray).map(_.asInstanceOf[Object]).asJava + } finally { + data.rdd.unpersist(blocking = false) + } + } + + /** + * Java stub for Python mllib GaussianMixtureModel.predictSoft() + */ + def predictSoftGMM( + data: JavaRDD[Vector], + wt: Object, + mu: Array[Object], + si: Array[Object]): RDD[Array[Double]] = { + + val weight = wt.asInstanceOf[Array[Double]] + 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) + } + /** * A Wrapper of MatrixFactorizationModel to provide helpfer method for Python */ diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 6b713aa39374e..f6b97abb1723c 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -15,19 +15,22 @@ # limitations under the License. # +from numpy import array + +from pyspark import RDD from pyspark import SparkContext from pyspark.mllib.common import callMLlibFunc, callJavaFunc -from pyspark.mllib.linalg import SparseVector, _convert_to_vector +from pyspark.mllib.linalg import DenseVector, SparseVector, _convert_to_vector +from pyspark.mllib.stat.distribution import MultivariateGaussian -__all__ = ['KMeansModel', 'KMeans'] +__all__ = ['KMeansModel', 'KMeans', 'GaussianMixtureModel', 'GaussianMixture'] class KMeansModel(object): """A clustering model derived from the k-means method. - >>> from numpy import array - >>> data = array([0.0,0.0, 1.0,1.0, 9.0,8.0, 8.0,9.0]).reshape(4,2) + >>> data = array([0.0,0.0, 1.0,1.0, 9.0,8.0, 8.0,9.0]).reshape(4, 2) >>> model = KMeans.train( ... sc.parallelize(data), 2, maxIterations=10, runs=30, initializationMode="random") >>> model.predict(array([0.0, 0.0])) == model.predict(array([1.0, 1.0])) @@ -86,6 +89,87 @@ def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||" return KMeansModel([c.toArray() for c in centers]) +class GaussianMixtureModel(object): + + """A clustering model derived from the Gaussian Mixture Model method. + + >>> clusterdata_1 = sc.parallelize(array([-0.1,-0.05,-0.01,-0.1, + ... 0.9,0.8,0.75,0.935, + ... -0.83,-0.68,-0.91,-0.76 ]).reshape(6, 2)) + >>> model = GaussianMixture.train(clusterdata_1, 3, convergenceTol=0.0001, + ... maxIterations=50, seed=10) + >>> labels = model.predict(clusterdata_1).collect() + >>> labels[0]==labels[1] + False + >>> labels[1]==labels[2] + True + >>> labels[4]==labels[5] + True + >>> clusterdata_2 = sc.parallelize(array([-5.1971, -2.5359, -3.8220, + ... -5.2211, -5.0602, 4.7118, + ... 6.8989, 3.4592, 4.6322, + ... 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) + >>> labels = model.predict(clusterdata_2).collect() + >>> labels[0]==labels[1]==labels[2] + True + >>> labels[3]==labels[4] + True + """ + + def __init__(self, weights, gaussians): + self.weights = weights + self.gaussians = gaussians + self.k = len(self.weights) + + def predict(self, x): + """ + Find the cluster to which the points in 'x' has maximum membership + in this model. + + :param x: RDD of data points. + :return: cluster_labels. RDD of cluster labels. + """ + if isinstance(x, RDD): + cluster_labels = self.predictSoft(x).map(lambda z: z.index(max(z))) + return cluster_labels + + def predictSoft(self, x): + """ + Find the membership of each point in 'x' to all mixture components. + + :param x: RDD of data points. + :return: membership_matrix. RDD of array of double values. + """ + 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 + + +class GaussianMixture(object): + """ + Estimate model parameters with the expectation-maximization algorithm. + + :param data: RDD of data points + :param k: Number of components + :param convergenceTol: Threshold value to check the convergence criteria. Defaults to 1e-3 + :param maxIterations: Number of iterations. Default to 100 + :param seed: Random Seed + """ + @classmethod + def train(cls, rdd, k, convergenceTol=1e-3, maxIterations=100, seed=None): + """Train a Gaussian Mixture clustering model.""" + weight, mu, sigma = callMLlibFunc("trainGaussianMixture", + rdd.map(_convert_to_vector), k, + convergenceTol, maxIterations, seed) + mvg_obj = [MultivariateGaussian(mu[i], sigma[i]) for i in range(k)] + return GaussianMixtureModel(weight, mvg_obj) + + def _test(): import doctest globs = globals().copy() diff --git a/python/pyspark/mllib/stat/__init__.py b/python/pyspark/mllib/stat/__init__.py index 799d260c096b1..b686d955a0080 100644 --- a/python/pyspark/mllib/stat/__init__.py +++ b/python/pyspark/mllib/stat/__init__.py @@ -20,5 +20,6 @@ """ from pyspark.mllib.stat._statistics import * +from pyspark.mllib.stat.distribution import MultivariateGaussian -__all__ = ["Statistics", "MultivariateStatisticalSummary"] +__all__ = ["Statistics", "MultivariateStatisticalSummary", "MultivariateGaussian"] diff --git a/python/pyspark/mllib/stat/distribution.py b/python/pyspark/mllib/stat/distribution.py new file mode 100644 index 0000000000000..07792e1532046 --- /dev/null +++ b/python/pyspark/mllib/stat/distribution.py @@ -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. +# + +from collections import namedtuple + +__all__ = ['MultivariateGaussian'] + + +class MultivariateGaussian(namedtuple('MultivariateGaussian', ['mu', 'sigma'])): + + """ 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.]])) + >>> (m[0], m[1]) + (DenseVector([11.0, 12.0]), array([[ 1., 5.],[ 3., 2.]])) + """ diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 61e0cf5d90bd0..42aa22873772d 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -167,6 +167,32 @@ def test_kmeans_deterministic(self): # TODO: Allow small numeric difference. self.assertTrue(array_equal(c1, c2)) + def test_gmm(self): + from pyspark.mllib.clustering import GaussianMixture + data = self.sc.parallelize([ + [1, 2], + [8, 9], + [-4, -3], + [-6, -7], + ]) + clusters = GaussianMixture.train(data, 2, convergenceTol=0.001, + maxIterations=100, seed=56) + labels = clusters.predict(data).collect() + self.assertEquals(labels[0], labels[1]) + self.assertEquals(labels[2], labels[3]) + + def test_gmm_deterministic(self): + from pyspark.mllib.clustering import GaussianMixture + x = range(0, 100, 10) + y = range(0, 100, 10) + data = self.sc.parallelize([[a, b] for a, b in zip(x, y)]) + clusters1 = GaussianMixture.train(data, 5, convergenceTol=0.001, + maxIterations=100, seed=63) + clusters2 = GaussianMixture.train(data, 5, convergenceTol=0.001, + maxIterations=100, seed=63) + for c1, c2 in zip(clusters1.weights, clusters2.weights): + self.assertEquals(round(c1, 7), round(c2, 7)) + def test_classification(self): from pyspark.mllib.classification import LogisticRegressionWithSGD, SVMWithSGD, NaiveBayes from pyspark.mllib.tree import DecisionTree, RandomForest, GradientBoostedTrees From 13531dd97c08563e53dacdaeaf1102bdd13ef825 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 2 Feb 2015 23:30:44 -0800 Subject: [PATCH 035/578] [SPARK-5501][SPARK-5420][SQL] Write support for the data source API This PR aims to support `INSERT INTO/OVERWRITE TABLE tableName` and `CREATE TABLE tableName AS SELECT` for the data source API (partitioned tables are not supported). In this PR, I am also adding the support of `IF NOT EXISTS` for our ddl parser. The current semantic of `IF NOT EXISTS` is explained as follows. * For a `CREATE TEMPORARY TABLE` statement, it does not `IF NOT EXISTS` for now. * For a `CREATE TABLE` statement (we are creating a metastore table), if there is an existing table having the same name ... * when `IF NOT EXISTS` clause is used, we will do nothing. * when `IF NOT EXISTS` clause is not used, the user will see an exception saying the table already exists. TODOs: - [x] CTAS support - [x] Programmatic APIs - [ ] Python API (another PR) - [x] More unit tests - [ ] Documents (another PR) marmbrus liancheng rxin Author: Yin Huai Closes #4294 from yhuai/writeSupport and squashes the following commits: 3db1539 [Yin Huai] save does not take overwrite. 1c98881 [Yin Huai] Fix test. 142372a [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupport 34e1bfb [Yin Huai] Address comments. 1682ca6 [Yin Huai] Better support for CTAS statements. e789d64 [Yin Huai] For the Scala API, let users to use tuples to provide options. 0128065 [Yin Huai] Short hand versions of save and load. 66ebd74 [Yin Huai] Formatting. 9203ec2 [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupport e5d29f2 [Yin Huai] Programmatic APIs. 1a719a5 [Yin Huai] CREATE TEMPORARY TABLE with IF NOT EXISTS is not allowed for now. 909924f [Yin Huai] Add saveAsTable for the data source API to DataFrame. 95a7c71 [Yin Huai] Fix bug when handling IF NOT EXISTS clause in a CREATE TEMPORARY TABLE statement. d37b19c [Yin Huai] Cheng's comments. fd6758c [Yin Huai] Use BeforeAndAfterAll. 7880891 [Yin Huai] Support CREATE TABLE AS SELECT STATEMENT and the IF NOT EXISTS clause. cb85b05 [Yin Huai] Initial write support. 2f91354 [Yin Huai] Make INSERT OVERWRITE/INTO statements consistent between HiveQL and SqlParser. --- .../apache/spark/sql/catalyst/SqlParser.scala | 4 +- .../org/apache/spark/sql/DataFrame.scala | 49 ++++- .../org/apache/spark/sql/DataFrameImpl.scala | 58 +++++- .../apache/spark/sql/IncomputableColumn.scala | 22 +++ .../scala/org/apache/spark/sql/SQLConf.scala | 6 + .../org/apache/spark/sql/SQLContext.scala | 26 ++- .../main/scala/org/apache/spark/sql/api.scala | 27 +++ .../spark/sql/execution/SparkStrategies.scala | 31 ++- .../apache/spark/sql/json/JSONRelation.scala | 59 +++++- .../sql/sources/DataSourceStrategy.scala | 9 +- .../apache/spark/sql/sources/commands.scala | 35 ++++ .../org/apache/spark/sql/sources/ddl.scala | 126 ++++++++++-- .../apache/spark/sql/sources/interfaces.scala | 23 ++- .../spark/sql/parquet/ParquetQuerySuite.scala | 12 +- .../sources/CreateTableAsSelectSuite.scala | 147 ++++++++++++++ .../spark/sql/sources/InsertIntoSuite.scala | 96 +++++++++ .../spark/sql/sources/SaveLoadSuite.scala | 88 +++++++++ .../apache/spark/sql/hive/HiveContext.scala | 67 ++++++- .../spark/sql/hive/HiveMetastoreCatalog.scala | 35 +++- .../spark/sql/hive/HiveStrategies.scala | 18 +- .../spark/sql/hive/execution/commands.scala | 76 ++++++- .../sql/hive/MetastoreDataSourcesSuite.scala | 185 +++++++++++++++++- .../sql/hive/execution/SQLQuerySuite.scala | 2 +- 23 files changed, 1141 insertions(+), 60 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/sources/InsertIntoSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.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 594a423146d77..25e639d390da0 100755 --- 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 @@ -147,8 +147,8 @@ class SqlParser extends AbstractSparkSQLParser { } protected lazy val insert: Parser[LogicalPlan] = - INSERT ~> OVERWRITE.? ~ (INTO ~> relation) ~ select ^^ { - case o ~ r ~ s => InsertIntoTable(r, Map.empty[String, Option[String]], s, o.isDefined) + INSERT ~> (OVERWRITE ^^^ true | INTO ^^^ false) ~ (TABLE ~> relation) ~ select ^^ { + case o ~ r ~ s => InsertIntoTable(r, Map.empty[String, Option[String]], s, o) } protected lazy val projection: Parser[Expression] = 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 385e1ec74f5f7..4cbfb6af5de94 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,8 +17,6 @@ package org.apache.spark.sql -import java.util.{List => JList} - import scala.reflect.ClassTag import org.apache.spark.annotation.{DeveloperApi, Experimental} @@ -485,6 +483,53 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { @Experimental override def saveAsTable(tableName: String): Unit + /** + * :: 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. + * + * 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 + override def saveAsTable( + tableName: String, + dataSourceName: String, + option: (String, String), + options: (String, String)*): Unit + + /** + * :: 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. + * + * 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 + override def saveAsTable( + tableName: String, + dataSourceName: String, + options: java.util.Map[String, String]): Unit + + @Experimental + override def save(path: String): Unit + + @Experimental + override def save( + dataSourceName: String, + option: (String, String), + options: (String, String)*): Unit + + @Experimental + override def save( + dataSourceName: String, + options: java.util.Map[String, String]): Unit + /** * :: Experimental :: * Adds the rows from this RDD to the specified table, optionally overwriting the existing data. 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 f8fcc25569482..f84dbf32fa5b2 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 @@ -36,6 +36,7 @@ import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} import org.apache.spark.sql.catalyst.plans.logical._ 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.types.{NumericType, StructType} import org.apache.spark.util.Utils @@ -303,8 +304,61 @@ private[sql] class DataFrameImpl protected[sql]( } override def saveAsTable(tableName: String): Unit = { - sqlContext.executePlan( - CreateTableAsSelect(None, tableName, logicalPlan, allowExisting = false)).toRdd + 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 = { + val cmd = + CreateTableUsingAsLogicalPlan( + tableName, + dataSourceName, + temporary = false, + (option +: options).toMap, + allowExisting = false, + 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:_*) } 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 2f8c695d5654b..9b051de68feb8 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 @@ -152,6 +152,28 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten 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() + + override def save( + dataSourceName: String, + options: java.util.Map[String, String]): Unit = err() + override def insertInto(tableName: String, overwrite: Boolean): Unit = err() override def toJSON: RDD[String] = 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 243dc997078f3..561a91d2d60ee 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,9 @@ private[spark] object SQLConf { // This is only used for the thriftserver 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" + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" } @@ -155,6 +158,9 @@ private[sql] class SQLConf extends Serializable { private[spark] def broadcastTimeout: Int = getConf(BROADCAST_TIMEOUT, (5 * 60).toString).toInt + private[spark] def defaultDataSourceName: String = + getConf(DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.parquet") + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ 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 f4692b3ff59d3..a741d0031d155 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -21,6 +21,7 @@ import java.beans.Introspector import java.util.Properties import scala.collection.immutable +import scala.collection.JavaConversions._ import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag @@ -37,7 +38,7 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution._ import org.apache.spark.sql.json._ import org.apache.spark.sql.jdbc.{JDBCPartition, JDBCPartitioningInfo, JDBCRelation} -import org.apache.spark.sql.sources.{LogicalRelation, BaseRelation, DDLParser, DataSourceStrategy} +import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -335,6 +336,29 @@ class SQLContext(@transient val sparkContext: SparkContext) applySchema(rowRDD, appliedSchema) } + @Experimental + def load(path: String): DataFrame = { + val dataSourceName = conf.defaultDataSourceName + load(dataSourceName, ("path", path)) + } + + @Experimental + def load( + dataSourceName: String, + option: (String, String), + options: (String, String)*): DataFrame = { + val resolved = ResolvedDataSource(this, None, dataSourceName, (option +: options).toMap) + DataFrame(this, LogicalRelation(resolved.relation)) + } + + @Experimental + def load( + dataSourceName: String, + options: java.util.Map[String, String]): DataFrame = { + val opts = options.toSeq + load(dataSourceName, opts.head, opts.tail:_*) + } + /** * :: Experimental :: * Construct an RDD representing the database table accessible via JDBC URL diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api.scala b/sql/core/src/main/scala/org/apache/spark/sql/api.scala index eb0eb3f32560c..c4a00cdb20408 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api.scala @@ -170,6 +170,33 @@ private[sql] trait DataFrameSpecificApi { @Experimental def saveAsTable(tableName: String): Unit + @Experimental + def saveAsTable( + tableName: String, + dataSourceName: String, + option: (String, String), + options: (String, String)*): Unit + + @Experimental + def saveAsTable( + tableName: String, + dataSourceName: String, + options: java.util.Map[String, String]): Unit + + @Experimental + def save(path: String): Unit + + @Experimental + def save( + dataSourceName: String, + option: (String, String), + options: (String, String)*): Unit + + @Experimental + def save( + dataSourceName: String, + options: java.util.Map[String, String]): Unit + @Experimental def insertInto(tableName: String, overwrite: Boolean): Unit 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 0cc9d049c9640..ff0609d4b3b72 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 @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} import org.apache.spark.sql.parquet._ import org.apache.spark.sql.types._ -import org.apache.spark.sql.sources.{CreateTempTableUsing, CreateTableUsing} +import org.apache.spark.sql.sources._ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { @@ -314,12 +314,33 @@ 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, options) => + case CreateTableUsing(tableName, userSpecifiedSchema, provider, true, opts, false) => ExecutedCommand( - CreateTempTableUsing(tableName, userSpecifiedSchema, provider, options)) :: Nil - - case CreateTableUsing(tableName, userSpecifiedSchema, provider, false, options) => + CreateTempTableUsing( + tableName, userSpecifiedSchema, provider, opts)) :: Nil + case c: CreateTableUsing if !c.temporary => + sys.error("Tables created with SQLContext must be TEMPORARY. Use a HiveContext instead.") + 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) => + val logicalPlan = sqlContext.parseSql(query) + val cmd = + CreateTempTableUsingAsSelect(tableName, provider, 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) => + val cmd = + CreateTempTableUsingAsSelect(tableName, provider, 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 _ => Nil } 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 1af96c28d5fdf..8372decbf8aa1 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 @@ -17,21 +17,26 @@ package org.apache.spark.sql.json -import org.apache.spark.sql.SQLContext +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.sources._ import org.apache.spark.sql.types.StructType -private[sql] class DefaultSource extends RelationProvider with SchemaRelationProvider { +private[sql] class DefaultSource + extends RelationProvider with SchemaRelationProvider with CreateableRelationProvider { /** Returns a new base relation with the parameters. */ override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - val fileName = parameters.getOrElse("path", sys.error("Option 'path' not specified")) + val path = parameters.getOrElse("path", sys.error("Option 'path' not specified")) val samplingRatio = parameters.get("samplingRatio").map(_.toDouble).getOrElse(1.0) - JSONRelation(fileName, samplingRatio, None)(sqlContext) + JSONRelation(path, samplingRatio, None)(sqlContext) } /** Returns a new base relation with the given schema and parameters. */ @@ -39,21 +44,37 @@ private[sql] class DefaultSource extends RelationProvider with SchemaRelationPro sqlContext: SQLContext, parameters: Map[String, String], schema: StructType): BaseRelation = { - val fileName = parameters.getOrElse("path", sys.error("Option 'path' not specified")) + val path = parameters.getOrElse("path", sys.error("Option 'path' not specified")) val samplingRatio = parameters.get("samplingRatio").map(_.toDouble).getOrElse(1.0) - JSONRelation(fileName, samplingRatio, Some(schema))(sqlContext) + JSONRelation(path, samplingRatio, Some(schema))(sqlContext) + } + + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String], + data: DataFrame): BaseRelation = { + val path = parameters.getOrElse("path", sys.error("Option 'path' not specified")) + val filesystemPath = new Path(path) + val fs = filesystemPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) + if (fs.exists(filesystemPath)) { + sys.error(s"path $path already exists.") + } + data.toJSON.saveAsTextFile(path) + + createRelation(sqlContext, parameters, data.schema) } } private[sql] case class JSONRelation( - fileName: String, + path: String, samplingRatio: Double, userSpecifiedSchema: Option[StructType])( @transient val sqlContext: SQLContext) - extends TableScan { + extends TableScan with InsertableRelation { - private def baseRDD = sqlContext.sparkContext.textFile(fileName) + // TODO: Support partitioned JSON relation. + private def baseRDD = sqlContext.sparkContext.textFile(path) override val schema = userSpecifiedSchema.getOrElse( JsonRDD.nullTypeToStringType( @@ -64,4 +85,24 @@ private[sql] case class JSONRelation( override def buildScan() = JsonRDD.jsonStringToRow(baseRDD, schema, sqlContext.conf.columnNameOfCorruptRecord) + + override def insert(data: DataFrame, overwrite: Boolean) = { + val filesystemPath = new Path(path) + val fs = filesystemPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) + + if (overwrite) { + try { + fs.delete(filesystemPath, true) + } catch { + case e: IOException => + throw new IOException( + s"Unable to clear output directory ${filesystemPath.toString} prior" + + s" to INSERT OVERWRITE a JSON table:\n${e.toString}") + } + data.toJSON.saveAsTextFile(path) + } else { + // TODO: Support INSERT INTO + sys.error("JSON table only support INSERT OVERWRITE for now.") + } + } } 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 d13f2ce2a5e1d..386ff2452f1a3 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 @@ -22,7 +22,7 @@ import org.apache.spark.sql.{Row, Strategy} 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.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, InsertIntoTable => LogicalInsertIntoTable} import org.apache.spark.sql.execution /** @@ -54,6 +54,13 @@ private[sql] object DataSourceStrategy extends Strategy { case l @ LogicalRelation(t: TableScan) => execution.PhysicalRDD(l.output, t.buildScan()) :: Nil + case i @ LogicalInsertIntoTable( + 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.") + } + execution.ExecutedCommand(InsertIntoRelation(t, query, overwrite)) :: Nil + case _ => Nil } 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 new file mode 100644 index 0000000000000..d7942dc30934b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.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.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.RunnableCommand + +private[sql] case class InsertIntoRelation( + relation: InsertableRelation, + query: LogicalPlan, + overwrite: Boolean) + extends RunnableCommand { + + override def run(sqlContext: SQLContext) = { + relation.insert(DataFrame(sqlContext, query), overwrite) + + Seq.empty[Row] + } +} 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 b1bbe0f89af73..ead827728cf4b 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 @@ -36,6 +36,7 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { try { Some(apply(input)) } catch { + case ddlException: DDLException => throw ddlException case _ if !exceptionOnError => None case x: Throwable => throw x } @@ -45,8 +46,7 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { lexical.initialize(reservedWords) phrase(dataType)(new lexical.Scanner(input)) match { case Success(r, x) => r - case x => - sys.error(s"Unsupported dataType: $x") + case x => throw new DDLException(s"Unsupported dataType: $x") } } @@ -56,8 +56,12 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { protected val CREATE = Keyword("CREATE") protected val TEMPORARY = Keyword("TEMPORARY") protected val TABLE = Keyword("TABLE") + protected val IF = Keyword("IF") + protected val NOT = Keyword("NOT") + protected val EXISTS = Keyword("EXISTS") protected val USING = Keyword("USING") protected val OPTIONS = Keyword("OPTIONS") + protected val AS = Keyword("AS") protected val COMMENT = Keyword("COMMENT") // Data types. @@ -83,22 +87,51 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { protected def start: Parser[LogicalPlan] = ddl /** - * `CREATE [TEMPORARY] TABLE avroTable + * `CREATE [TEMPORARY] TABLE avroTable [IF NOT EXISTS] * USING org.apache.spark.sql.avro * OPTIONS (path "../hive/src/test/resources/data/files/episodes.avro")` * or - * `CREATE [TEMPORARY] TABLE avroTable(intField int, stringField string...) + * `CREATE [TEMPORARY] TABLE avroTable(intField int, stringField string...) [IF NOT EXISTS] * USING org.apache.spark.sql.avro * OPTIONS (path "../hive/src/test/resources/data/files/episodes.avro")` + * or + * `CREATE [TEMPORARY] TABLE avroTable [IF NOT EXISTS] + * USING org.apache.spark.sql.avro + * OPTIONS (path "../hive/src/test/resources/data/files/episodes.avro")` + * AS SELECT ... */ protected lazy val createTable: Parser[LogicalPlan] = ( - (CREATE ~> TEMPORARY.? <~ TABLE) ~ ident - ~ (tableCols).? ~ (USING ~> className) ~ (OPTIONS ~> options) ^^ { - case temp ~ tableName ~ columns ~ provider ~ opts => - val userSpecifiedSchema = columns.flatMap(fields => Some(StructType(fields))) - CreateTableUsing(tableName, userSpecifiedSchema, provider, temp.isDefined, opts) - } + (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( + "a CREATE TEMPORARY TABLE statement does not allow IF NOT EXISTS clause.") + } + + if (query.isDefined) { + if (columns.isDefined) { + throw new DDLException( + "a CREATE TABLE AS SELECT statement does not allow column definitions.") + } + CreateTableUsingAsSelect(tableName, + provider, + temp.isDefined, + opts, + allowExisting.isDefined, + query.get) + } else { + val userSpecifiedSchema = columns.flatMap(fields => Some(StructType(fields))) + CreateTableUsing( + tableName, + userSpecifiedSchema, + provider, + temp.isDefined, + opts, + allowExisting.isDefined) + } + } ) protected lazy val tableCols: Parser[Seq[StructField]] = "(" ~> repsep(column, ",") <~ ")" @@ -193,7 +226,7 @@ object ResolvedDataSource { dataSource .asInstanceOf[org.apache.spark.sql.sources.SchemaRelationProvider] .createRelation(sqlContext, new CaseInsensitiveMap(options), schema) - case _ => + case dataSource: org.apache.spark.sql.sources.RelationProvider => sys.error(s"${clazz.getCanonicalName} does not allow user-specified schemas.") } } @@ -203,7 +236,7 @@ object ResolvedDataSource { dataSource .asInstanceOf[org.apache.spark.sql.sources.RelationProvider] .createRelation(sqlContext, new CaseInsensitiveMap(options)) - case _ => + case dataSource: org.apache.spark.sql.sources.SchemaRelationProvider => sys.error(s"A schema needs to be specified when using ${clazz.getCanonicalName}.") } } @@ -211,6 +244,32 @@ object ResolvedDataSource { new ResolvedDataSource(clazz, relation) } + + def apply( + sqlContext: SQLContext, + provider: String, + 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 { + case dataSource: org.apache.spark.sql.sources.CreateableRelationProvider => + dataSource + .asInstanceOf[org.apache.spark.sql.sources.CreateableRelationProvider] + .createRelation(sqlContext, options, data) + case _ => + sys.error(s"${clazz.getCanonicalName} does not allow create table as select.") + } + + new ResolvedDataSource(clazz, relation) + } } private[sql] case class ResolvedDataSource(provider: Class[_], relation: BaseRelation) @@ -220,13 +279,30 @@ private[sql] case class CreateTableUsing( userSpecifiedSchema: Option[StructType], provider: String, temporary: Boolean, - options: Map[String, String]) extends Command + options: Map[String, String], + allowExisting: Boolean) extends Command + +private[sql] case class CreateTableUsingAsSelect( + tableName: String, + provider: String, + temporary: Boolean, + options: Map[String, String], + allowExisting: Boolean, + query: String) extends Command + +private[sql] case class CreateTableUsingAsLogicalPlan( + tableName: String, + provider: String, + temporary: Boolean, + options: Map[String, String], + allowExisting: Boolean, + query: LogicalPlan) extends Command private [sql] case class CreateTempTableUsing( tableName: String, userSpecifiedSchema: Option[StructType], provider: String, - options: Map[String, String]) extends RunnableCommand { + options: Map[String, String]) extends RunnableCommand { def run(sqlContext: SQLContext) = { val resolved = ResolvedDataSource(sqlContext, userSpecifiedSchema, provider, options) @@ -236,6 +312,22 @@ private [sql] case class CreateTempTableUsing( } } +private [sql] case class CreateTempTableUsingAsSelect( + tableName: String, + provider: String, + options: Map[String, String], + query: LogicalPlan) extends RunnableCommand { + + def run(sqlContext: SQLContext) = { + val df = DataFrame(sqlContext, query) + val resolved = ResolvedDataSource(sqlContext, provider, options, df) + sqlContext.registerRDDAsTable( + DataFrame(sqlContext, LogicalRelation(resolved.relation)), tableName) + + Seq.empty + } +} + /** * Builds a map in which keys are case insensitive */ @@ -253,3 +345,9 @@ protected class CaseInsensitiveMap(map: Map[String, String]) extends Map[String, override def -(key: String): Map[String, String] = baseMap - key.toLowerCase() } + +/** + * The exception thrown from the DDL parser. + * @param message + */ +protected[sql] class DDLException(message: String) extends Exception(message) 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 cd82cc6ecb61b..ad0a35b91ebc2 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.{Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Row, SQLContext} import org.apache.spark.sql.catalyst.expressions.{Expression, Attribute} import org.apache.spark.sql.types.StructType @@ -77,6 +77,14 @@ trait SchemaRelationProvider { schema: StructType): BaseRelation } +@DeveloperApi +trait CreateableRelationProvider { + def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String], + data: DataFrame): BaseRelation +} + /** * ::DeveloperApi:: * Represents a collection of tuples with a known schema. Classes that extend BaseRelation must @@ -108,7 +116,7 @@ abstract class BaseRelation { * A BaseRelation that can produce all of its tuples as an RDD of Row objects. */ @DeveloperApi -abstract class TableScan extends BaseRelation { +trait TableScan extends BaseRelation { def buildScan(): RDD[Row] } @@ -118,7 +126,7 @@ abstract class TableScan extends BaseRelation { * containing all of its tuples as Row objects. */ @DeveloperApi -abstract class PrunedScan extends BaseRelation { +trait PrunedScan extends BaseRelation { def buildScan(requiredColumns: Array[String]): RDD[Row] } @@ -132,7 +140,7 @@ abstract class PrunedScan extends BaseRelation { * as filtering partitions based on a bloom filter. */ @DeveloperApi -abstract class PrunedFilteredScan extends BaseRelation { +trait PrunedFilteredScan extends BaseRelation { def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] } @@ -145,6 +153,11 @@ abstract class PrunedFilteredScan extends BaseRelation { * for experimentation. */ @Experimental -abstract class CatalystScan extends BaseRelation { +trait CatalystScan extends BaseRelation { def buildScan(requiredColumns: Seq[Attribute], filters: Seq[Expression]): RDD[Row] } + +@DeveloperApi +trait InsertableRelation extends BaseRelation { + def insert(data: DataFrame, overwrite: Boolean): Unit +} 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 3d82f4bce7778..5ec7a156d9353 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 @@ -37,11 +37,21 @@ class ParquetQuerySuite extends QueryTest with ParquetTest { test("appending") { val data = (0 until 10).map(i => (i, i.toString)) withParquetTable(data, "t") { - sql("INSERT INTO t SELECT * FROM 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("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("self-join") { // 4 rows, cells of column 1 of row 2 and row 4 are null val data = (1 to 4).map { i => 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 new file mode 100644 index 0000000000000..b02389978b625 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources + +import java.io.File + +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.sql.catalyst.util +import org.apache.spark.util.Utils + +class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll { + + import caseInsensisitiveContext._ + + var path: File = null + + override def beforeAll(): Unit = { + path = util.getTempFilePath("jsonCTAS").getCanonicalFile + val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) + jsonRDD(rdd).registerTempTable("jt") + } + + override def afterAll(): Unit = { + dropTempTable("jt") + } + + after { + if (path.exists()) Utils.deleteRecursively(path) + } + + test("CREATE TEMPORARY TABLE AS SELECT") { + sql( + s""" + |CREATE TEMPORARY TABLE jsonTable + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${path.toString}' + |) AS + |SELECT a, b FROM jt + """.stripMargin) + + checkAnswer( + sql("SELECT a, b FROM jsonTable"), + sql("SELECT a, b FROM jt").collect()) + + dropTempTable("jsonTable") + } + + test("create a table, drop it and create another one with the same name") { + sql( + s""" + |CREATE TEMPORARY TABLE jsonTable + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${path.toString}' + |) AS + |SELECT a, b FROM jt + """.stripMargin) + + checkAnswer( + sql("SELECT a, b FROM jsonTable"), + sql("SELECT a, b FROM jt").collect()) + + dropTempTable("jsonTable") + + val message = intercept[RuntimeException]{ + sql( + s""" + |CREATE TEMPORARY TABLE jsonTable + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${path.toString}' + |) AS + |SELECT a * 4 FROM jt + """.stripMargin) + }.getMessage + assert( + message.contains(s"path ${path.toString} already exists."), + "CREATE TEMPORARY TABLE IF NOT EXISTS should not be allowed.") + + // Explicitly delete it. + if (path.exists()) Utils.deleteRecursively(path) + + 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") + } + + test("CREATE TEMPORARY TABLE AS SELECT with IF NOT EXISTS is not allowed") { + val message = intercept[DDLException]{ + sql( + s""" + |CREATE TEMPORARY TABLE IF NOT EXISTS jsonTable + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${path.toString}' + |) AS + |SELECT b FROM jt + """.stripMargin) + }.getMessage + assert( + message.contains("a CREATE TEMPORARY TABLE statement does not allow IF NOT EXISTS clause."), + "CREATE TEMPORARY TABLE IF NOT EXISTS should not be allowed.") + } + + test("a CTAS statement with column definitions is not allowed") { + intercept[DDLException]{ + sql( + s""" + |CREATE TEMPORARY TABLE jsonTable (a int, b string) + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${path.toString}' + |) AS + |SELECT a, b FROM jt + """.stripMargin) + } + } +} 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/InsertIntoSuite.scala new file mode 100644 index 0000000000000..f91cea6a37060 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertIntoSuite.scala @@ -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. + */ + +package org.apache.spark.sql.sources + +import java.io.File + +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.util +import org.apache.spark.util.Utils + +class InsertIntoSuite extends DataSourceTest with BeforeAndAfterAll { + + import caseInsensisitiveContext._ + + var path: File = null + + override def beforeAll: Unit = { + path = util.getTempFilePath("jsonCTAS").getCanonicalFile + val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}""")) + jsonRDD(rdd).registerTempTable("jt") + sql( + s""" + |CREATE TEMPORARY TABLE jsonTable (a int, b string) + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${path.toString}' + |) + """.stripMargin) + } + + override def afterAll: Unit = { + dropTempTable("jsonTable") + dropTempTable("jt") + if (path.exists()) Utils.deleteRecursively(path) + } + + test("Simple INSERT OVERWRITE a JSONRelation") { + sql( + s""" + |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt + """.stripMargin) + + checkAnswer( + sql("SELECT a, b FROM jsonTable"), + (1 to 10).map(i => Row(i, s"str$i")) + ) + } + + test("INSERT OVERWRITE a JSONRelation multiple times") { + sql( + s""" + |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt + """.stripMargin) + + sql( + s""" + |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt + """.stripMargin) + + sql( + s""" + |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt + """.stripMargin) + + checkAnswer( + sql("SELECT a, b FROM jsonTable"), + (1 to 10).map(i => Row(i, s"str$i")) + ) + } + + test("INSERT INTO not supported for JSONRelation for now") { + intercept[RuntimeException]{ + sql( + s""" + |INSERT INTO TABLE jsonTable SELECT a, b FROM jt + """.stripMargin) + } + } +} 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 new file mode 100644 index 0000000000000..fe2f76cc397f5 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 org.scalatest.BeforeAndAfterAll + +import org.apache.spark.sql.DataFrame +import org.apache.spark.util.Utils + +import org.apache.spark.sql.catalyst.util + +class SaveLoadSuite extends DataSourceTest with BeforeAndAfterAll { + + import caseInsensisitiveContext._ + + var originalDefaultSource: String = null + + var path: File = null + + var df: DataFrame = null + + 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) + } + + override def afterAll(): Unit = { + conf.setConf("spark.sql.default.datasource", originalDefaultSource) + } + + after { + if (path.exists()) Utils.deleteRecursively(path) + } + + def checkLoad(): Unit = { + checkAnswer(load(path.toString), df.collect()) + checkAnswer(load("org.apache.spark.sql.json", ("path", path.toString)), df.collect()) + } + + test("save with overwrite and load") { + df.save(path.toString) + checkLoad + } + + test("save with data source and options, and load") { + df.save("org.apache.spark.sql.json", ("path", path.toString)) + checkLoad + } + + test("save and save again") { + df.save(path.toString) + + val message = intercept[RuntimeException] { + df.save(path.toString) + }.getMessage + + assert( + message.contains("already exists"), + "We should complain that the path already exists.") + + if (path.exists()) Utils.deleteRecursively(path) + + df.save(path.toString) + checkLoad + } +} \ 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 5efc3b1e30774..f6d9027f90a99 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 @@ -41,7 +41,7 @@ import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateAnalysisOperat 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.DataSourceStrategy +import org.apache.spark.sql.sources.{CreateTableUsing, DataSourceStrategy} import org.apache.spark.sql.types._ /** @@ -86,6 +86,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { * @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) } @@ -106,6 +107,70 @@ 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/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index d910ee950904d..48bea6c1bd685 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 @@ -23,10 +23,9 @@ 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.TableType -import org.apache.hadoop.hive.metastore.api.{Table => TTable, Partition => TPartition, FieldSchema} -import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table, HiveException} -import org.apache.hadoop.hive.ql.metadata.InvalidTableException +import org.apache.hadoop.hive.metastore.{Warehouse, TableType} +import org.apache.hadoop.hive.metastore.api.{Table => TTable, Partition => TPartition, AlreadyExistsException, FieldSchema} +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} @@ -52,6 +51,8 @@ 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) + protected[hive] lazy val hiveWarehouse = new Warehouse(hive.hiveconf) + // 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) { @@ -99,11 +100,22 @@ 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 + */ def createDataSourceTable( tableName: String, userSpecifiedSchema: Option[StructType], provider: String, - options: Map[String, String]) = { + options: Map[String, String], + isExternal: Boolean): Unit = { val (dbName, tblName) = processDatabaseAndTableName("default", tableName) val tbl = new Table(dbName, tblName) @@ -113,8 +125,13 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } options.foreach { case (key, value) => tbl.setSerdeParam(key, value) } - tbl.setProperty("EXTERNAL", "TRUE") - tbl.setTableType(TableType.EXTERNAL_TABLE) + if (isExternal) { + tbl.setProperty("EXTERNAL", "TRUE") + tbl.setTableType(TableType.EXTERNAL_TABLE) + } else { + tbl.setProperty("EXTERNAL", "FALSE") + tbl.setTableType(TableType.MANAGED_TABLE) + } // create the table synchronized { @@ -122,6 +139,10 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } } + def hiveDefaultTableFilePath(tableName: String): String = { + hiveWarehouse.getTablePath(client.getDatabaseCurrent, tableName).toString + } + def tableExists(tableIdentifier: Seq[String]): Boolean = { val tableIdent = processTableIdentifier(tableIdentifier) val databaseName = tableIdent.lift(tableIdent.size - 2).getOrElse( 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 fa997288a2848..d89111094b9ff 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 @@ -33,7 +33,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.CreateTableUsing +import org.apache.spark.sql.sources.{CreateTableUsingAsLogicalPlan, CreateTableUsingAsSelect, CreateTableUsing} import org.apache.spark.sql.types.StringType @@ -212,9 +212,21 @@ private[hive] trait HiveStrategies { object HiveDDLStrategy extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case CreateTableUsing(tableName, userSpecifiedSchema, provider, false, options) => + case CreateTableUsing(tableName, userSpecifiedSchema, provider, false, opts, allowExisting) => ExecutedCommand( - CreateMetastoreDataSource(tableName, userSpecifiedSchema, provider, options)) :: Nil + CreateMetastoreDataSource( + tableName, userSpecifiedSchema, provider, opts, allowExisting)) :: Nil + + case CreateTableUsingAsSelect(tableName, provider, false, opts, allowExisting, query) => + val logicalPlan = hiveContext.parseSql(query) + val cmd = + CreateMetastoreDataSourceAsSelect(tableName, provider, opts, allowExisting, logicalPlan) + ExecutedCommand(cmd) :: Nil + + case CreateTableUsingAsLogicalPlan(tableName, provider, false, opts, allowExisting, query) => + val cmd = + CreateMetastoreDataSourceAsSelect(tableName, provider, opts, allowExisting, 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 4814cb7ebfe51..95dcaccefdc54 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,8 +18,10 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.sources.ResolvedDataSource +import org.apache.spark.sql.{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 import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.types.StructType @@ -102,11 +104,77 @@ case class CreateMetastoreDataSource( tableName: String, userSpecifiedSchema: Option[StructType], provider: String, - options: Map[String, String]) extends RunnableCommand { + options: Map[String, String], + allowExisting: Boolean) extends RunnableCommand { - override def run(sqlContext: SQLContext) = { + override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] - hiveContext.catalog.createDataSourceTable(tableName, userSpecifiedSchema, provider, options) + + if (hiveContext.catalog.tableExists(tableName :: Nil)) { + if (allowExisting) { + return Seq.empty[Row] + } else { + sys.error(s"Table $tableName already exists.") + } + } + + var isExternal = true + val optionsWithPath = + if (!options.contains("path")) { + isExternal = false + options + ("path" -> hiveContext.catalog.hiveDefaultTableFilePath(tableName)) + } else { + options + } + + hiveContext.catalog.createDataSourceTable( + tableName, + userSpecifiedSchema, + provider, + optionsWithPath, + isExternal) + + Seq.empty[Row] + } +} + +case class CreateMetastoreDataSourceAsSelect( + tableName: String, + provider: String, + 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 isExternal = true + val optionsWithPath = + if (!options.contains("path")) { + isExternal = false + options + ("path" -> hiveContext.catalog.hiveDefaultTableFilePath(tableName)) + } else { + options + } + + // Create the relation based on the data of df. + ResolvedDataSource(sqlContext, provider, optionsWithPath, df) + + hiveContext.catalog.createDataSourceTable( + tableName, + None, + provider, + optionsWithPath, + isExternal) 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 7408c7ffd69e8..85795acb658e2 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 @@ -22,7 +22,9 @@ import java.io.File import org.scalatest.BeforeAndAfterEach import org.apache.commons.io.FileUtils +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.catalyst.util import org.apache.spark.sql._ import org.apache.spark.util.Utils import org.apache.spark.sql.types._ @@ -36,9 +38,11 @@ import org.apache.spark.sql.hive.test.TestHive._ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { override def afterEach(): Unit = { reset() + if (ctasPath.exists()) Utils.deleteRecursively(ctasPath) } val filePath = Utils.getSparkClassLoader.getResource("sample.json").getFile + var ctasPath: File = util.getTempFilePath("jsonCTAS").getCanonicalFile test ("persistent JSON table") { sql( @@ -94,7 +98,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { StructField("", innerStruct, true) :: StructField("b", StringType, true) :: Nil) - assert(expectedSchema == table("jsonTable").schema) + assert(expectedSchema === table("jsonTable").schema) jsonFile(filePath).registerTempTable("expectedJsonTable") @@ -137,6 +141,11 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { intercept[Exception] { sql("SELECT * FROM jsonTable").collect() } + + assert( + (new File(filePath)).exists(), + "The table with specified path is considered as an external table, " + + "its data should not deleted after DROP TABLE.") } test("check change without refresh") { @@ -240,7 +249,144 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { invalidateTable("jsonTable") val expectedSchema = StructType(StructField("c_!@(3)", IntegerType, true) :: Nil) - assert(expectedSchema == table("jsonTable").schema) + assert(expectedSchema === table("jsonTable").schema) + } + + test("CTAS") { + sql( + s""" + |CREATE TABLE jsonTable + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${filePath}' + |) + """.stripMargin) + + sql( + s""" + |CREATE TABLE ctasJsonTable + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${ctasPath}' + |) AS + |SELECT * FROM jsonTable + """.stripMargin) + + assert(table("ctasJsonTable").schema === table("jsonTable").schema) + + checkAnswer( + sql("SELECT * FROM ctasJsonTable"), + sql("SELECT * FROM jsonTable").collect()) + } + + test("CTAS with IF NOT EXISTS") { + sql( + s""" + |CREATE TABLE jsonTable + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${filePath}' + |) + """.stripMargin) + + sql( + s""" + |CREATE TABLE ctasJsonTable + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${ctasPath}' + |) AS + |SELECT * FROM jsonTable + """.stripMargin) + + // Create the table again should trigger a AlreadyExistsException. + val message = intercept[RuntimeException] { + sql( + s""" + |CREATE TABLE ctasJsonTable + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${ctasPath}' + |) AS + |SELECT * FROM jsonTable + """.stripMargin) + }.getMessage + assert(message.contains("Table ctasJsonTable already exists."), + "We should complain that ctasJsonTable already exists") + + // The following statement should be fine if it has IF NOT EXISTS. + // It tries to create a table ctasJsonTable with a new schema. + // The actual table's schema and data should not be changed. + sql( + s""" + |CREATE TABLE IF NOT EXISTS ctasJsonTable + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${ctasPath}' + |) AS + |SELECT a FROM jsonTable + """.stripMargin) + + // Discard the cached relation. + invalidateTable("ctasJsonTable") + + // Schema should not be changed. + assert(table("ctasJsonTable").schema === table("jsonTable").schema) + // Table data should not be changed. + checkAnswer( + sql("SELECT * FROM ctasJsonTable"), + sql("SELECT * FROM jsonTable").collect()) + } + + test("CTAS a managed table") { + sql( + s""" + |CREATE TABLE jsonTable + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${filePath}' + |) + """.stripMargin) + + new Path("/Users/yhuai/Desktop/whatever") + + + val expectedPath = catalog.hiveDefaultTableFilePath("ctasJsonTable") + val filesystemPath = new Path(expectedPath) + val fs = filesystemPath.getFileSystem(sparkContext.hadoopConfiguration) + if (fs.exists(filesystemPath)) fs.delete(filesystemPath, true) + + // It is a managed table when we do not specify the location. + sql( + s""" + |CREATE TABLE ctasJsonTable + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | + |) AS + |SELECT * FROM jsonTable + """.stripMargin) + + assert(fs.exists(filesystemPath), s"$expectedPath should exist after we create the table.") + + sql( + s""" + |CREATE TABLE loadedTable + |USING org.apache.spark.sql.json.DefaultSource + |OPTIONS ( + | path '${expectedPath}' + |) + """.stripMargin) + + assert(table("ctasJsonTable").schema === table("loadedTable").schema) + + checkAnswer( + sql("SELECT * FROM ctasJsonTable"), + sql("SELECT * FROM loadedTable").collect() + ) + + sql("DROP TABLE ctasJsonTable") + assert(!fs.exists(filesystemPath), s"$expectedPath should not exist after we drop the table.") } test("SPARK-5286 Fail to drop an invalid table when using the data source API") { @@ -255,4 +401,39 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach { sql("DROP TABLE jsonTable").collect().foreach(println) } + + test("save and load 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) + + df.saveAsTable("savedJsonTable") + + checkAnswer( + sql("SELECT * FROM savedJsonTable"), + df.collect()) + + createTable("createdJsonTable", catalog.hiveDefaultTableFilePath("savedJsonTable"), false) + assert(table("createdJsonTable").schema === df.schema) + checkAnswer( + sql("SELECT * FROM createdJsonTable"), + df.collect()) + + val message = intercept[RuntimeException] { + createTable("createdJsonTable", filePath.toString, false) + }.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) + checkAnswer( + sql("SELECT * FROM createdJsonTable"), + df.collect()) + + conf.setConf("spark.sql.default.datasource", originalDefaultSource) + } } 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 eb7a7750af02d..4efe0c5e0cd44 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 @@ -170,7 +170,7 @@ class SQLQuerySuite extends QueryTest { sql("CREATE TABLE test2 (key INT, value STRING)") testData.insertInto("test2") testData.insertInto("test2") - sql("SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key").saveAsTable("test") + 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) From b8ebebeaaa259be4fcddf65b3280d23165b011a1 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 2 Feb 2015 23:35:07 -0800 Subject: [PATCH 036/578] [SPARK-5414] Add SparkFirehoseListener class for consuming all SparkListener events There isn't a good way to write a SparkListener that receives all SparkListener events and which will be future-compatible (e.g. it will receive events introduced in newer versions of Spark without having to override new methods to process those events). To address this, this patch adds `SparkFirehoseListener`, a SparkListener implementation that receives all events and dispatches them to a single `onEvent` method (which can be overridden by users). Author: Josh Rosen Closes #4210 from JoshRosen/firehose-listener and squashes the following commits: 223f579 [Josh Rosen] Expand comment to explain rationale for this being a Java class. ecdfaed [Josh Rosen] Add SparkFirehoseListener class for consuming all SparkListener events. --- .../apache/spark/SparkFirehoseListener.java | 115 ++++++++++++++++++ 1 file changed, 115 insertions(+) create mode 100644 core/src/main/java/org/apache/spark/SparkFirehoseListener.java diff --git a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java new file mode 100644 index 0000000000000..fbc5666959055 --- /dev/null +++ b/core/src/main/java/org/apache/spark/SparkFirehoseListener.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; + +import org.apache.spark.scheduler.*; + +/** + * Class that allows users to receive all SparkListener events. + * Users should override the onEvent method. + * + * This is a concrete Java class in order to ensure that we don't forget to update it when adding + * new methods to SparkListener: forgetting to add a method will result in a compilation error (if + * this was a concrete Scala class, default implementations of new event handlers would be inherited + * from the SparkListener trait). + */ +public class SparkFirehoseListener implements SparkListener { + + public void onEvent(SparkListenerEvent event) { } + + @Override + public final void onStageCompleted(SparkListenerStageCompleted stageCompleted) { + onEvent(stageCompleted); + } + + @Override + public final void onStageSubmitted(SparkListenerStageSubmitted stageSubmitted) { + onEvent(stageSubmitted); + } + + @Override + public final void onTaskStart(SparkListenerTaskStart taskStart) { + onEvent(taskStart); + } + + @Override + public final void onTaskGettingResult(SparkListenerTaskGettingResult taskGettingResult) { + onEvent(taskGettingResult); + } + + @Override + public final void onTaskEnd(SparkListenerTaskEnd taskEnd) { + onEvent(taskEnd); + } + + @Override + public final void onJobStart(SparkListenerJobStart jobStart) { + onEvent(jobStart); + } + + @Override + public final void onJobEnd(SparkListenerJobEnd jobEnd) { + onEvent(jobEnd); + } + + @Override + public final void onEnvironmentUpdate(SparkListenerEnvironmentUpdate environmentUpdate) { + onEvent(environmentUpdate); + } + + @Override + public final void onBlockManagerAdded(SparkListenerBlockManagerAdded blockManagerAdded) { + onEvent(blockManagerAdded); + } + + @Override + public final void onBlockManagerRemoved(SparkListenerBlockManagerRemoved blockManagerRemoved) { + onEvent(blockManagerRemoved); + } + + @Override + public final void onUnpersistRDD(SparkListenerUnpersistRDD unpersistRDD) { + onEvent(unpersistRDD); + } + + @Override + public final void onApplicationStart(SparkListenerApplicationStart applicationStart) { + onEvent(applicationStart); + } + + @Override + public final void onApplicationEnd(SparkListenerApplicationEnd applicationEnd) { + onEvent(applicationEnd); + } + + @Override + public final void onExecutorMetricsUpdate( + SparkListenerExecutorMetricsUpdate executorMetricsUpdate) { + onEvent(executorMetricsUpdate); + } + + @Override + public final void onExecutorAdded(SparkListenerExecutorAdded executorAdded) { + onEvent(executorAdded); + } + + @Override + public final void onExecutorRemoved(SparkListenerExecutorRemoved executorRemoved) { + onEvent(executorRemoved); + } +} From 0cc7b88c99405db99bc4c3d66f5409e5da0e3c6e Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 2 Feb 2015 23:49:09 -0800 Subject: [PATCH 037/578] [SPARK-5536] replace old ALS implementation by the new one The only issue is that `analyzeBlock` is removed, which was marked as a developer API. I didn't change other tests in the ALSSuite under `spark.mllib` to ensure that the implementation is correct. CC: srowen coderxiang Author: Xiangrui Meng Closes #4321 from mengxr/SPARK-5536 and squashes the following commits: 5a3cee8 [Xiangrui Meng] update python tests that are too strict e840acf [Xiangrui Meng] ignore scala style check for ALS.train e9a721c [Xiangrui Meng] update mima excludes 9ee6a36 [Xiangrui Meng] merge master 9a8aeac [Xiangrui Meng] update tests d8c3271 [Xiangrui Meng] remove analyzeBlocks d68eee7 [Xiangrui Meng] add checkpoint to new ALS 22a56f8 [Xiangrui Meng] wrap old ALS c387dff [Xiangrui Meng] support random seed 3bdf24b [Xiangrui Meng] make storage level configurable in the new ALS --- .../apache/spark/ml/recommendation/ALS.scala | 69 +- .../spark/mllib/recommendation/ALS.scala | 600 +----------------- .../spark/ml/recommendation/ALSSuite.scala | 2 +- .../spark/mllib/recommendation/ALSSuite.scala | 18 - project/MimaExcludes.scala | 7 +- python/pyspark/mllib/recommendation.py | 16 +- 6 files changed, 90 insertions(+), 622 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 82d21d5e4cb6e..511cb2fe4005e 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 @@ -22,6 +22,7 @@ import java.{util => ju} import scala.collection.mutable import scala.reflect.ClassTag import scala.util.Sorting +import scala.util.hashing.byteswap64 import com.github.fommil.netlib.BLAS.{getInstance => blas} import com.github.fommil.netlib.LAPACK.{getInstance => lapack} @@ -37,6 +38,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame import org.apache.spark.sql.Dsl._ import org.apache.spark.sql.types.{DoubleType, FloatType, IntegerType, StructField, StructType} +import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils import org.apache.spark.util.collection.{OpenHashMap, OpenHashSet, SortDataFormat, Sorter} import org.apache.spark.util.random.XORShiftRandom @@ -412,7 +414,7 @@ object ALS extends Logging { /** * Implementation of the ALS algorithm. */ - def train[ID: ClassTag]( + def train[ID: ClassTag]( // scalastyle:ignore ratings: RDD[Rating[ID]], rank: Int = 10, numUserBlocks: Int = 10, @@ -421,34 +423,47 @@ object ALS extends Logging { regParam: Double = 1.0, implicitPrefs: Boolean = false, alpha: Double = 1.0, - nonnegative: Boolean = false)( + nonnegative: Boolean = false, + intermediateRDDStorageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK, + finalRDDStorageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK, + seed: Long = 0L)( implicit ord: Ordering[ID]): (RDD[(ID, Array[Float])], RDD[(ID, Array[Float])]) = { + 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 userLocalIndexEncoder = new LocalIndexEncoder(userPart.numPartitions) val itemLocalIndexEncoder = new LocalIndexEncoder(itemPart.numPartitions) val solver = if (nonnegative) new NNLSSolver else new CholeskySolver - val blockRatings = partitionRatings(ratings, userPart, itemPart).cache() - val (userInBlocks, userOutBlocks) = makeBlocks("user", blockRatings, userPart, itemPart) + val blockRatings = partitionRatings(ratings, userPart, itemPart) + .persist(intermediateRDDStorageLevel) + val (userInBlocks, userOutBlocks) = + makeBlocks("user", blockRatings, userPart, itemPart, intermediateRDDStorageLevel) // materialize blockRatings and user blocks userOutBlocks.count() val swappedBlockRatings = blockRatings.map { case ((userBlockId, itemBlockId), RatingBlock(userIds, itemIds, localRatings)) => ((itemBlockId, userBlockId), RatingBlock(itemIds, userIds, localRatings)) } - val (itemInBlocks, itemOutBlocks) = makeBlocks("item", swappedBlockRatings, itemPart, userPart) + val (itemInBlocks, itemOutBlocks) = + makeBlocks("item", swappedBlockRatings, itemPart, userPart, intermediateRDDStorageLevel) // materialize item blocks itemOutBlocks.count() - var userFactors = initialize(userInBlocks, rank) - var itemFactors = initialize(itemInBlocks, rank) + val seedGen = new XORShiftRandom(seed) + var userFactors = initialize(userInBlocks, rank, seedGen.nextLong()) + var itemFactors = initialize(itemInBlocks, rank, seedGen.nextLong()) if (implicitPrefs) { for (iter <- 1 to maxIter) { - userFactors.setName(s"userFactors-$iter").persist() + userFactors.setName(s"userFactors-$iter").persist(intermediateRDDStorageLevel) val previousItemFactors = itemFactors itemFactors = computeFactors(userFactors, userOutBlocks, itemInBlocks, rank, regParam, userLocalIndexEncoder, implicitPrefs, alpha, solver) previousItemFactors.unpersist() - itemFactors.setName(s"itemFactors-$iter").persist() + if (sc.checkpointDir.isDefined && (iter % 3 == 0)) { + itemFactors.checkpoint() + } + itemFactors.setName(s"itemFactors-$iter").persist(intermediateRDDStorageLevel) val previousUserFactors = userFactors userFactors = computeFactors(itemFactors, itemOutBlocks, userInBlocks, rank, regParam, itemLocalIndexEncoder, implicitPrefs, alpha, solver) @@ -467,21 +482,23 @@ object ALS extends Logging { .join(userFactors) .values .setName("userFactors") - .cache() - userIdAndFactors.count() - itemFactors.unpersist() + .persist(finalRDDStorageLevel) val itemIdAndFactors = itemInBlocks .mapValues(_.srcIds) .join(itemFactors) .values .setName("itemFactors") - .cache() - itemIdAndFactors.count() - userInBlocks.unpersist() - userOutBlocks.unpersist() - itemInBlocks.unpersist() - itemOutBlocks.unpersist() - blockRatings.unpersist() + .persist(finalRDDStorageLevel) + if (finalRDDStorageLevel != StorageLevel.NONE) { + userIdAndFactors.count() + itemFactors.unpersist() + itemIdAndFactors.count() + userInBlocks.unpersist() + userOutBlocks.unpersist() + itemInBlocks.unpersist() + itemOutBlocks.unpersist() + blockRatings.unpersist() + } val userOutput = userIdAndFactors.flatMap { case (ids, factors) => ids.view.zip(factors) } @@ -546,14 +563,15 @@ object ALS extends Logging { */ private def initialize[ID]( inBlocks: RDD[(Int, InBlock[ID])], - rank: Int): RDD[(Int, FactorBlock)] = { + rank: Int, + seed: Long): RDD[(Int, FactorBlock)] = { // Choose a unit vector uniformly at random from the unit sphere, but from the // "first quadrant" where all elements are nonnegative. This can be done by choosing // elements distributed as Normal(0,1) and taking the absolute value, and then normalizing. // This appears to create factorizations that have a slightly better reconstruction // (<1%) compared picking elements uniformly at random in [0,1]. inBlocks.map { case (srcBlockId, inBlock) => - val random = new XORShiftRandom(srcBlockId) + val random = new XORShiftRandom(byteswap64(seed ^ srcBlockId)) val factors = Array.fill(inBlock.srcIds.length) { val factor = Array.fill(rank)(random.nextGaussian().toFloat) val nrm = blas.snrm2(rank, factor, 1) @@ -877,7 +895,8 @@ object ALS extends Logging { prefix: String, ratingBlocks: RDD[((Int, Int), RatingBlock[ID])], srcPart: Partitioner, - dstPart: Partitioner)( + dstPart: Partitioner, + storageLevel: StorageLevel)( implicit srcOrd: Ordering[ID]): (RDD[(Int, InBlock[ID])], RDD[(Int, OutBlock)]) = { val inBlocks = ratingBlocks.map { case ((srcBlockId, dstBlockId), RatingBlock(srcIds, dstIds, ratings)) => @@ -914,7 +933,8 @@ object ALS extends Logging { builder.add(dstBlockId, srcIds, dstLocalIndices, ratings) } builder.build().compress() - }.setName(prefix + "InBlocks").cache() + }.setName(prefix + "InBlocks") + .persist(storageLevel) val outBlocks = inBlocks.mapValues { case InBlock(srcIds, dstPtrs, dstEncodedIndices, _) => val encoder = new LocalIndexEncoder(dstPart.numPartitions) val activeIds = Array.fill(dstPart.numPartitions)(mutable.ArrayBuilder.make[Int]) @@ -936,7 +956,8 @@ object ALS extends Logging { activeIds.map { x => x.result() } - }.setName(prefix + "OutBlocks").cache() + }.setName(prefix + "OutBlocks") + .persist(storageLevel) (inBlocks, outBlocks) } 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 a5ffe888ca880..f4f51f2ac5210 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 @@ -17,46 +17,12 @@ package org.apache.spark.mllib.recommendation -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer -import scala.math.{abs, sqrt} -import scala.util.{Random, Sorting} -import scala.util.hashing.byteswap32 - -import org.jblas.{DoubleMatrix, SimpleBlas, Solve} - -import org.apache.spark.{HashPartitioner, Logging, Partitioner} -import org.apache.spark.SparkContext._ +import org.apache.spark.Logging import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.mllib.optimization.NNLS +import org.apache.spark.ml.recommendation.{ALS => NewALS} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.Utils - -/** - * Out-link information for a user or product block. This includes the original user/product IDs - * of the elements within this block, and the list of destination blocks that each user or - * product will need to send its feature vector to. - */ -private[recommendation] -case class OutLinkBlock(elementIds: Array[Int], shouldSend: Array[mutable.BitSet]) - - -/** - * In-link information for a user (or product) block. This includes the original user/product IDs - * of the elements within this block, as well as an array of indices and ratings that specify - * which user in the block will be rated by which products from each product block (or vice-versa). - * Specifically, if this InLinkBlock is for users, ratingsForBlock(b)(i) will contain two arrays, - * indices and ratings, for the i'th product that will be sent to us by product block b (call this - * P). These arrays represent the users that product P had ratings for (by their index in this - * block), as well as the corresponding rating for each one. We can thus use this information when - * we get product block b's message to update the corresponding users. - */ -private[recommendation] case class InLinkBlock( - elementIds: Array[Int], ratingsForBlock: Array[Array[(Array[Int], Array[Double])]]) - /** * :: Experimental :: @@ -201,6 +167,8 @@ class ALS private ( */ @DeveloperApi def setIntermediateRDDStorageLevel(storageLevel: StorageLevel): this.type = { + require(storageLevel != StorageLevel.NONE, + "ALS is not designed to run without persisting intermediate RDDs.") this.intermediateRDDStorageLevel = storageLevel this } @@ -236,431 +204,39 @@ class ALS private ( this.numProductBlocks } - val userPartitioner = new ALSPartitioner(numUserBlocks) - val productPartitioner = new ALSPartitioner(numProductBlocks) - - val ratingsByUserBlock = ratings.map { rating => - (userPartitioner.getPartition(rating.user), rating) - } - val ratingsByProductBlock = ratings.map { rating => - (productPartitioner.getPartition(rating.product), - Rating(rating.product, rating.user, rating.rating)) - } - - val (userInLinks, userOutLinks) = - makeLinkRDDs(numUserBlocks, numProductBlocks, ratingsByUserBlock, productPartitioner) - val (productInLinks, productOutLinks) = - makeLinkRDDs(numProductBlocks, numUserBlocks, ratingsByProductBlock, userPartitioner) - userInLinks.setName("userInLinks") - userOutLinks.setName("userOutLinks") - productInLinks.setName("productInLinks") - productOutLinks.setName("productOutLinks") - - // Initialize user and product factors randomly, but use a deterministic seed for each - // partition so that fault recovery works - val seedGen = new Random(seed) - val seed1 = seedGen.nextInt() - val seed2 = seedGen.nextInt() - var users = userOutLinks.mapPartitionsWithIndex { (index, itr) => - val rand = new Random(byteswap32(seed1 ^ index)) - itr.map { case (x, y) => - (x, y.elementIds.map(_ => randomFactor(rank, rand))) - } - } - var products = productOutLinks.mapPartitionsWithIndex { (index, itr) => - val rand = new Random(byteswap32(seed2 ^ index)) - itr.map { case (x, y) => - (x, y.elementIds.map(_ => randomFactor(rank, rand))) - } - } - - if (implicitPrefs) { - for (iter <- 1 to iterations) { - // perform ALS update - logInfo("Re-computing I given U (Iteration %d/%d)".format(iter, iterations)) - // Persist users because it will be called twice. - users.setName(s"users-$iter").persist() - val YtY = Some(sc.broadcast(computeYtY(users))) - val previousProducts = products - products = updateFeatures(numProductBlocks, users, userOutLinks, productInLinks, - rank, lambda, alpha, YtY) - previousProducts.unpersist() - logInfo("Re-computing U given I (Iteration %d/%d)".format(iter, iterations)) - if (sc.checkpointDir.isDefined && (iter % 3 == 0)) { - products.checkpoint() - } - products.setName(s"products-$iter").persist() - val XtX = Some(sc.broadcast(computeYtY(products))) - val previousUsers = users - users = updateFeatures(numUserBlocks, products, productOutLinks, userInLinks, - rank, lambda, alpha, XtX) - previousUsers.unpersist() - } - } else { - for (iter <- 1 to iterations) { - // perform ALS update - logInfo("Re-computing I given U (Iteration %d/%d)".format(iter, iterations)) - products = updateFeatures(numProductBlocks, users, userOutLinks, productInLinks, - rank, lambda, alpha, YtY = None) - if (sc.checkpointDir.isDefined && (iter % 3 == 0)) { - products.checkpoint() - } - products.setName(s"products-$iter") - logInfo("Re-computing U given I (Iteration %d/%d)".format(iter, iterations)) - users = updateFeatures(numUserBlocks, products, productOutLinks, userInLinks, - rank, lambda, alpha, YtY = None) - users.setName(s"users-$iter") - } + val (floatUserFactors, floatProdFactors) = NewALS.train[Int]( + ratings = ratings.map(r => NewALS.Rating(r.user, r.product, r.rating.toFloat)), + rank = rank, + numUserBlocks = numUserBlocks, + numItemBlocks = numProductBlocks, + maxIter = iterations, + regParam = lambda, + implicitPrefs = implicitPrefs, + alpha = alpha, + nonnegative = nonnegative, + intermediateRDDStorageLevel = intermediateRDDStorageLevel, + finalRDDStorageLevel = StorageLevel.NONE, + seed = seed) + + val userFactors = floatUserFactors + .mapValues(_.map(_.toDouble)) + .setName("users") + .persist(finalRDDStorageLevel) + val prodFactors = floatProdFactors + .mapValues(_.map(_.toDouble)) + .setName("products") + .persist(finalRDDStorageLevel) + if (finalRDDStorageLevel != StorageLevel.NONE) { + userFactors.count() + prodFactors.count() } - - // The last `products` will be used twice. One to generate the last `users` and the other to - // generate `productsOut`. So we cache it for better performance. - products.setName("products").persist() - - // Flatten and cache the two final RDDs to un-block them - val usersOut = unblockFactors(users, userOutLinks) - val productsOut = unblockFactors(products, productOutLinks) - - usersOut.setName("usersOut").persist(finalRDDStorageLevel) - productsOut.setName("productsOut").persist(finalRDDStorageLevel) - - // Materialize usersOut and productsOut. - usersOut.count() - productsOut.count() - - products.unpersist() - - // Clean up. - userInLinks.unpersist() - userOutLinks.unpersist() - productInLinks.unpersist() - productOutLinks.unpersist() - - new MatrixFactorizationModel(rank, usersOut, productsOut) + new MatrixFactorizationModel(rank, userFactors, prodFactors) } /** * Java-friendly version of [[ALS.run]]. */ def run(ratings: JavaRDD[Rating]): MatrixFactorizationModel = run(ratings.rdd) - - /** - * Computes the (`rank x rank`) matrix `YtY`, where `Y` is the (`nui x rank`) matrix of factors - * for each user (or product), in a distributed fashion. - * - * @param factors the (block-distributed) user or product factor vectors - * @return YtY - whose value is only used in the implicit preference model - */ - private def computeYtY(factors: RDD[(Int, Array[Array[Double]])]) = { - val n = rank * (rank + 1) / 2 - val LYtY = factors.values.aggregate(new DoubleMatrix(n))( seqOp = (L, Y) => { - Y.foreach(y => dspr(1.0, wrapDoubleArray(y), L)) - L - }, combOp = (L1, L2) => { - L1.addi(L2) - }) - val YtY = new DoubleMatrix(rank, rank) - fillFullMatrix(LYtY, YtY) - YtY - } - - /** - * Adds alpha * x * x.t to a matrix in-place. This is the same as BLAS's DSPR. - * - * @param L the lower triangular part of the matrix packed in an array (row major) - */ - private def dspr(alpha: Double, x: DoubleMatrix, L: DoubleMatrix) = { - val n = x.length - var i = 0 - var j = 0 - var idx = 0 - var axi = 0.0 - val xd = x.data - val Ld = L.data - while (i < n) { - axi = alpha * xd(i) - j = 0 - while (j <= i) { - Ld(idx) += axi * xd(j) - j += 1 - idx += 1 - } - i += 1 - } - } - - /** - * Wrap a double array in a DoubleMatrix without creating garbage. - * This is a temporary fix for jblas 1.2.3; it should be safe to move back to the - * DoubleMatrix(double[]) constructor come jblas 1.2.4. - */ - private def wrapDoubleArray(v: Array[Double]): DoubleMatrix = { - new DoubleMatrix(v.length, 1, v: _*) - } - - /** - * Flatten out blocked user or product factors into an RDD of (id, factor vector) pairs - */ - private def unblockFactors( - blockedFactors: RDD[(Int, Array[Array[Double]])], - outLinks: RDD[(Int, OutLinkBlock)]): RDD[(Int, Array[Double])] = { - blockedFactors.join(outLinks).flatMap { case (b, (factors, outLinkBlock)) => - for (i <- 0 until factors.length) yield (outLinkBlock.elementIds(i), factors(i)) - } - } - - /** - * Make the out-links table for a block of the users (or products) dataset given the list of - * (user, product, rating) values for the users in that block (or the opposite for products). - */ - private def makeOutLinkBlock(numProductBlocks: Int, ratings: Array[Rating], - productPartitioner: Partitioner): OutLinkBlock = { - val userIds = ratings.map(_.user).distinct.sorted - val numUsers = userIds.length - val userIdToPos = userIds.zipWithIndex.toMap - val shouldSend = Array.fill(numUsers)(new mutable.BitSet(numProductBlocks)) - for (r <- ratings) { - shouldSend(userIdToPos(r.user))(productPartitioner.getPartition(r.product)) = true - } - OutLinkBlock(userIds, shouldSend) - } - - /** - * Make the in-links table for a block of the users (or products) dataset given a list of - * (user, product, rating) values for the users in that block (or the opposite for products). - */ - private def makeInLinkBlock(numProductBlocks: Int, ratings: Array[Rating], - productPartitioner: Partitioner): InLinkBlock = { - val userIds = ratings.map(_.user).distinct.sorted - val userIdToPos = userIds.zipWithIndex.toMap - // Split out our ratings by product block - val blockRatings = Array.fill(numProductBlocks)(new ArrayBuffer[Rating]) - for (r <- ratings) { - blockRatings(productPartitioner.getPartition(r.product)) += r - } - val ratingsForBlock = new Array[Array[(Array[Int], Array[Double])]](numProductBlocks) - for (productBlock <- 0 until numProductBlocks) { - // Create an array of (product, Seq(Rating)) ratings - val groupedRatings = blockRatings(productBlock).groupBy(_.product).toArray - // Sort them by product ID - val ordering = new Ordering[(Int, ArrayBuffer[Rating])] { - def compare(a: (Int, ArrayBuffer[Rating]), b: (Int, ArrayBuffer[Rating])): Int = - a._1 - b._1 - } - Sorting.quickSort(groupedRatings)(ordering) - // Translate the user IDs to indices based on userIdToPos - ratingsForBlock(productBlock) = groupedRatings.map { case (p, rs) => - (rs.view.map(r => userIdToPos(r.user)).toArray, rs.view.map(_.rating).toArray) - } - } - InLinkBlock(userIds, ratingsForBlock) - } - - /** - * Make RDDs of InLinkBlocks and OutLinkBlocks given an RDD of (blockId, (u, p, r)) values for - * the users (or (blockId, (p, u, r)) for the products). We create these simultaneously to avoid - * having to shuffle the (blockId, (u, p, r)) RDD twice, or to cache it. - */ - private def makeLinkRDDs( - numUserBlocks: Int, - numProductBlocks: Int, - ratingsByUserBlock: RDD[(Int, Rating)], - productPartitioner: Partitioner): (RDD[(Int, InLinkBlock)], RDD[(Int, OutLinkBlock)]) = { - val grouped = ratingsByUserBlock.partitionBy(new HashPartitioner(numUserBlocks)) - val links = grouped.mapPartitionsWithIndex((blockId, elements) => { - val ratings = elements.map(_._2).toArray - val inLinkBlock = makeInLinkBlock(numProductBlocks, ratings, productPartitioner) - val outLinkBlock = makeOutLinkBlock(numProductBlocks, ratings, productPartitioner) - Iterator.single((blockId, (inLinkBlock, outLinkBlock))) - }, preservesPartitioning = true) - val inLinks = links.mapValues(_._1) - val outLinks = links.mapValues(_._2) - inLinks.persist(intermediateRDDStorageLevel) - outLinks.persist(intermediateRDDStorageLevel) - (inLinks, outLinks) - } - - /** - * Make a random factor vector with the given random. - */ - private def randomFactor(rank: Int, rand: Random): Array[Double] = { - // Choose a unit vector uniformly at random from the unit sphere, but from the - // "first quadrant" where all elements are nonnegative. This can be done by choosing - // elements distributed as Normal(0,1) and taking the absolute value, and then normalizing. - // This appears to create factorizations that have a slightly better reconstruction - // (<1%) compared picking elements uniformly at random in [0,1]. - val factor = Array.fill(rank)(abs(rand.nextGaussian())) - val norm = sqrt(factor.map(x => x * x).sum) - factor.map(x => x / norm) - } - - /** - * Compute the user feature vectors given the current products (or vice-versa). This first joins - * the products with their out-links to generate a set of messages to each destination block - * (specifically, the features for the products that user block cares about), then groups these - * by destination and joins them with the in-link info to figure out how to update each user. - * It returns an RDD of new feature vectors for each user block. - */ - private def updateFeatures( - numUserBlocks: Int, - products: RDD[(Int, Array[Array[Double]])], - productOutLinks: RDD[(Int, OutLinkBlock)], - userInLinks: RDD[(Int, InLinkBlock)], - rank: Int, - lambda: Double, - alpha: Double, - YtY: Option[Broadcast[DoubleMatrix]]): RDD[(Int, Array[Array[Double]])] = { - productOutLinks.join(products).flatMap { case (bid, (outLinkBlock, factors)) => - val toSend = Array.fill(numUserBlocks)(new ArrayBuffer[Array[Double]]) - for (p <- 0 until outLinkBlock.elementIds.length; userBlock <- 0 until numUserBlocks) { - if (outLinkBlock.shouldSend(p)(userBlock)) { - toSend(userBlock) += factors(p) - } - } - toSend.zipWithIndex.map{ case (buf, idx) => (idx, (bid, buf.toArray)) } - }.groupByKey(new HashPartitioner(numUserBlocks)) - .join(userInLinks) - .mapValues{ case (messages, inLinkBlock) => - updateBlock(messages, inLinkBlock, rank, lambda, alpha, YtY) - } - } - - /** - * Compute the new feature vectors for a block of the users matrix given the list of factors - * it received from each product and its InLinkBlock. - */ - private def updateBlock(messages: Iterable[(Int, Array[Array[Double]])], inLinkBlock: InLinkBlock, - rank: Int, lambda: Double, alpha: Double, YtY: Option[Broadcast[DoubleMatrix]]) - : Array[Array[Double]] = - { - // Sort the incoming block factor messages by block ID and make them an array - val blockFactors = messages.toSeq.sortBy(_._1).map(_._2).toArray // Array[Array[Double]] - val numProductBlocks = blockFactors.length - val numUsers = inLinkBlock.elementIds.length - - // We'll sum up the XtXes using vectors that represent only the lower-triangular part, since - // the matrices are symmetric - val triangleSize = rank * (rank + 1) / 2 - val userXtX = Array.fill(numUsers)(DoubleMatrix.zeros(triangleSize)) - val userXy = Array.fill(numUsers)(DoubleMatrix.zeros(rank)) - - // Some temp variables to avoid memory allocation - val tempXtX = DoubleMatrix.zeros(triangleSize) - val fullXtX = DoubleMatrix.zeros(rank, rank) - - // Count the number of ratings each user gives to provide user-specific regularization - val numRatings = Array.fill(numUsers)(0) - - // Compute the XtX and Xy values for each user by adding products it rated in each product - // block - for (productBlock <- 0 until numProductBlocks) { - var p = 0 - while (p < blockFactors(productBlock).length) { - val x = wrapDoubleArray(blockFactors(productBlock)(p)) - tempXtX.fill(0.0) - dspr(1.0, x, tempXtX) - val (us, rs) = inLinkBlock.ratingsForBlock(productBlock)(p) - if (implicitPrefs) { - var i = 0 - while (i < us.length) { - numRatings(us(i)) += 1 - // Extension to the original paper to handle rs(i) < 0. confidence is a function - // of |rs(i)| instead so that it is never negative: - val confidence = 1 + alpha * abs(rs(i)) - SimpleBlas.axpy(confidence - 1.0, tempXtX, userXtX(us(i))) - // For rs(i) < 0, the corresponding entry in P is 0 now, not 1 -- negative rs(i) - // means we try to reconstruct 0. We add terms only where P = 1, so, term below - // is now only added for rs(i) > 0: - if (rs(i) > 0) { - SimpleBlas.axpy(confidence, x, userXy(us(i))) - } - i += 1 - } - } else { - var i = 0 - while (i < us.length) { - numRatings(us(i)) += 1 - userXtX(us(i)).addi(tempXtX) - SimpleBlas.axpy(rs(i), x, userXy(us(i))) - i += 1 - } - } - p += 1 - } - } - - val ws = if (nonnegative) NNLS.createWorkspace(rank) else null - - // Solve the least-squares problem for each user and return the new feature vectors - Array.range(0, numUsers).map { index => - // Compute the full XtX matrix from the lower-triangular part we got above - fillFullMatrix(userXtX(index), fullXtX) - // Add regularization - val regParam = numRatings(index) * lambda - var i = 0 - while (i < rank) { - fullXtX.data(i * rank + i) += regParam - i += 1 - } - // Solve the resulting matrix, which is symmetric and positive-definite - if (implicitPrefs) { - solveLeastSquares(fullXtX.addi(YtY.get.value), userXy(index), ws) - } else { - solveLeastSquares(fullXtX, userXy(index), ws) - } - } - } - - /** - * Given A^T A and A^T b, find the x minimising ||Ax - b||_2, possibly subject - * to nonnegativity constraints if `nonnegative` is true. - */ - private def solveLeastSquares(ata: DoubleMatrix, atb: DoubleMatrix, - ws: NNLS.Workspace): Array[Double] = { - if (!nonnegative) { - Solve.solvePositive(ata, atb).data - } else { - NNLS.solve(ata, atb, ws) - } - } - - /** - * Given a triangular matrix in the order of fillXtX above, compute the full symmetric square - * matrix that it represents, storing it into destMatrix. - */ - private def fillFullMatrix(triangularMatrix: DoubleMatrix, destMatrix: DoubleMatrix) { - val rank = destMatrix.rows - var i = 0 - var pos = 0 - while (i < rank) { - var j = 0 - while (j <= i) { - destMatrix.data(i*rank + j) = triangularMatrix.data(pos) - destMatrix.data(j*rank + i) = triangularMatrix.data(pos) - pos += 1 - j += 1 - } - i += 1 - } - } -} - -/** - * Partitioner for ALS. - */ -private[recommendation] class ALSPartitioner(override val numPartitions: Int) extends Partitioner { - override def getPartition(key: Any): Int = { - Utils.nonNegativeMod(byteswap32(key.asInstanceOf[Int]), numPartitions) - } - - override def equals(obj: Any): Boolean = { - obj match { - case p: ALSPartitioner => - this.numPartitions == p.numPartitions - case _ => - false - } - } } /** @@ -834,120 +410,4 @@ object ALS { : MatrixFactorizationModel = { trainImplicit(ratings, rank, iterations, 0.01, -1, 1.0) } - - /** - * :: DeveloperApi :: - * Statistics of a block in ALS computation. - * - * @param category type of this block, "user" or "product" - * @param index index of this block - * @param count number of users or products inside this block, the same as the number of - * least-squares problems to solve on this block in each iteration - * @param numRatings total number of ratings inside this block, the same as the number of outer - * products we need to make on this block in each iteration - * @param numInLinks total number of incoming links, the same as the number of vectors to retrieve - * before each iteration - * @param numOutLinks total number of outgoing links, the same as the number of vectors to send - * for the next iteration - */ - @DeveloperApi - case class BlockStats( - category: String, - index: Int, - count: Long, - numRatings: Long, - numInLinks: Long, - numOutLinks: Long) - - /** - * :: DeveloperApi :: - * Given an RDD of ratings, number of user blocks, and number of product blocks, computes the - * statistics of each block in ALS computation. This is useful for estimating cost and diagnosing - * load balance. - * - * @param ratings an RDD of ratings - * @param numUserBlocks number of user blocks - * @param numProductBlocks number of product blocks - * @return statistics of user blocks and product blocks - */ - @DeveloperApi - def analyzeBlocks( - ratings: RDD[Rating], - numUserBlocks: Int, - numProductBlocks: Int): Array[BlockStats] = { - - val userPartitioner = new ALSPartitioner(numUserBlocks) - val productPartitioner = new ALSPartitioner(numProductBlocks) - - val ratingsByUserBlock = ratings.map { rating => - (userPartitioner.getPartition(rating.user), rating) - } - val ratingsByProductBlock = ratings.map { rating => - (productPartitioner.getPartition(rating.product), - Rating(rating.product, rating.user, rating.rating)) - } - - val als = new ALS() - val (userIn, userOut) = - als.makeLinkRDDs(numUserBlocks, numProductBlocks, ratingsByUserBlock, userPartitioner) - val (prodIn, prodOut) = - als.makeLinkRDDs(numProductBlocks, numUserBlocks, ratingsByProductBlock, productPartitioner) - - def sendGrid(outLinks: RDD[(Int, OutLinkBlock)]): Map[(Int, Int), Long] = { - outLinks.map { x => - val grid = new mutable.HashMap[(Int, Int), Long]() - val uPartition = x._1 - x._2.shouldSend.foreach { ss => - ss.foreach { pPartition => - val pair = (uPartition, pPartition) - grid.put(pair, grid.getOrElse(pair, 0L) + 1L) - } - } - grid - }.reduce { (grid1, grid2) => - grid2.foreach { x => - grid1.put(x._1, grid1.getOrElse(x._1, 0L) + x._2) - } - grid1 - }.toMap - } - - val userSendGrid = sendGrid(userOut) - val prodSendGrid = sendGrid(prodOut) - - val userInbound = new Array[Long](numUserBlocks) - val prodInbound = new Array[Long](numProductBlocks) - val userOutbound = new Array[Long](numUserBlocks) - val prodOutbound = new Array[Long](numProductBlocks) - - for (u <- 0 until numUserBlocks; p <- 0 until numProductBlocks) { - userOutbound(u) += userSendGrid.getOrElse((u, p), 0L) - prodInbound(p) += userSendGrid.getOrElse((u, p), 0L) - userInbound(u) += prodSendGrid.getOrElse((p, u), 0L) - prodOutbound(p) += prodSendGrid.getOrElse((p, u), 0L) - } - - val userCounts = userOut.mapValues(x => x.elementIds.length).collectAsMap() - val prodCounts = prodOut.mapValues(x => x.elementIds.length).collectAsMap() - - val userRatings = countRatings(userIn) - val prodRatings = countRatings(prodIn) - - val userStats = Array.tabulate(numUserBlocks)( - u => BlockStats("user", u, userCounts(u), userRatings(u), userInbound(u), userOutbound(u))) - val productStatus = Array.tabulate(numProductBlocks)( - p => BlockStats("product", p, prodCounts(p), prodRatings(p), prodInbound(p), prodOutbound(p))) - - (userStats ++ productStatus).toArray - } - - private def countRatings(inLinks: RDD[(Int, InLinkBlock)]): Map[Int, Long] = { - inLinks.mapValues { ilb => - var numRatings = 0L - ilb.ratingsForBlock.foreach { ar => - ar.foreach { p => numRatings += p._1.length } - } - numRatings - }.collectAsMap().toMap - } } 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 ee08c3c32760e..acc447742bad0 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 @@ -414,7 +414,7 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext with Logging { val (training, test) = genExplicitTestData(numUsers = 20, numItems = 40, rank = 2, noiseStd = 0.01) for ((numUserBlocks, numItemBlocks) <- Seq((1, 1), (1, 2), (2, 1), (2, 2))) { - testALS(training, test, maxIter = 4, rank = 2, regParam = 0.01, targetRMSE = 0.03, + testALS(training, test, maxIter = 4, rank = 3, regParam = 0.01, targetRMSE = 0.03, numUserBlocks = numUserBlocks, numItemBlocks = numItemBlocks) } } 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 e9fc37e000526..8775c0ca9df84 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 @@ -24,9 +24,7 @@ import scala.util.Random import org.scalatest.FunSuite import org.jblas.DoubleMatrix -import org.apache.spark.SparkContext._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.mllib.recommendation.ALS.BlockStats import org.apache.spark.storage.StorageLevel object ALSSuite { @@ -189,22 +187,6 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext { testALS(100, 200, 2, 15, 0.7, 0.4, false, false, false, -1, -1, false) } - test("analyze one user block and one product block") { - val localRatings = Seq( - Rating(0, 100, 1.0), - Rating(0, 101, 2.0), - Rating(0, 102, 3.0), - Rating(1, 102, 4.0), - Rating(2, 103, 5.0)) - val ratings = sc.makeRDD(localRatings, 2) - val stats = ALS.analyzeBlocks(ratings, 1, 1) - assert(stats.size === 2) - assert(stats(0) === BlockStats("user", 0, 3, 5, 4, 3)) - assert(stats(1) === BlockStats("product", 0, 4, 5, 3, 4)) - } - - // TODO: add tests for analyzing multiple user/product blocks - /** * Test if we can correctly factorize R = U * P where U and P are of known rank. * diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 78de1f0652741..b17532c1d814c 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -69,7 +69,12 @@ object MimaExcludes { ) ++ Seq( // SPARK-5540 ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.solveLeastSquares") + "org.apache.spark.mllib.recommendation.ALS.solveLeastSquares"), + // SPARK-5536 + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$^dateFeatures"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$^dateBlock") ) ++ Seq( // SPARK-3325 ProblemFilters.exclude[MissingMethodProblem]( diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 97ec74eda0b71..0d99e6dedfad9 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -49,17 +49,17 @@ class MatrixFactorizationModel(JavaModelWrapper): >>> r3 = (2, 1, 2.0) >>> ratings = sc.parallelize([r1, r2, r3]) >>> model = ALS.trainImplicit(ratings, 1, seed=10) - >>> model.predict(2,2) - 0.4473... + >>> model.predict(2, 2) + 0.43... >>> testset = sc.parallelize([(1, 2), (1, 1)]) - >>> model = ALS.train(ratings, 1, seed=10) + >>> model = ALS.train(ratings, 2, seed=0) >>> model.predictAll(testset).collect() - [Rating(user=1, product=1, rating=1.0471...), Rating(user=1, product=2, rating=1.9679...)] + [Rating(user=1, product=1, rating=1.0...), Rating(user=1, product=2, rating=1.9...)] >>> model = ALS.train(ratings, 4, seed=10) >>> model.userFeatures().collect() - [(2, array('d', [...])), (1, array('d', [...]))] + [(1, array('d', [...])), (2, array('d', [...]))] >>> first_user = model.userFeatures().take(1)[0] >>> latents = first_user[1] @@ -67,7 +67,7 @@ class MatrixFactorizationModel(JavaModelWrapper): True >>> model.productFeatures().collect() - [(2, array('d', [...])), (1, array('d', [...]))] + [(1, array('d', [...])), (2, array('d', [...]))] >>> first_product = model.productFeatures().take(1)[0] >>> latents = first_product[1] @@ -76,11 +76,11 @@ class MatrixFactorizationModel(JavaModelWrapper): >>> model = ALS.train(ratings, 1, nonnegative=True, seed=10) >>> model.predict(2,2) - 3.735... + 3.8... >>> model = ALS.trainImplicit(ratings, 1, nonnegative=True, seed=10) >>> model.predict(2,2) - 0.4473... + 0.43... """ def predict(self, user, product): return self._java_model.predict(int(user), int(product)) From 980764f3c0c065cc32454a036e8d0ead5a92037b Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 2 Feb 2015 23:57:35 -0800 Subject: [PATCH 038/578] [SPARK-1405] [mllib] Latent Dirichlet Allocation (LDA) using EM MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit **This PR introduces an API + simple implementation for Latent Dirichlet Allocation (LDA).** The [design doc for this PR](https://docs.google.com/document/d/1kSsDqTeZMEB94Bs4GTd0mvdAmduvZSSkpoSfn-seAzo) has been updated since I initially posted it. In particular, see the API and Planning for the Future sections. * Settle on a public API which may eventually include: * more inference algorithms * more options / functionality * Have an initial easy-to-understand implementation which others may improve. * This is NOT intended to support every topic model out there. However, if there are suggestions for making this extensible or pluggable in the future, that could be nice, as long as it does not complicate the API or implementation too much. * This may not be very scalable currently. It will be important to check and improve accuracy. For correctness of the implementation, please check against the Asuncion et al. (2009) paper in the design doc. **Dependency: This makes MLlib depend on GraphX.** Files and classes: * LDA.scala (441 lines): * class LDA (main estimator class) * LDA.Document (text + document ID) * LDAModel.scala (266 lines) * abstract class LDAModel * class LocalLDAModel * class DistributedLDAModel * LDAExample.scala (245 lines): script to run LDA + a simple (private) Tokenizer * LDASuite.scala (144 lines) Data/model representation and algorithm: * Data/model: Uses GraphX, with term vertices + document vertices * Algorithm: EM, following [Asuncion, Welling, Smyth, and Teh. "On Smoothing and Inference for Topic Models." UAI, 2009.](http://arxiv-web3.library.cornell.edu/abs/1205.2662v1) * For more details, please see the description in the “DEVELOPERS NOTE” in LDA.scala Please refer to the JIRA for more discussion + the [design doc for this PR](https://docs.google.com/document/d/1kSsDqTeZMEB94Bs4GTd0mvdAmduvZSSkpoSfn-seAzo) Here, I list the main changes AFTER the design doc was posted. Design decisions: * logLikelihood() computes the log likelihood of the data and the current point estimate of parameters. This is different from the likelihood of the data given the hyperparameters, which would be harder to compute. I’d describe the current approach as more frequentist, whereas the harder approach would be more Bayesian. * The current API takes Documents as token count vectors. I believe there should be an extended API taking RDD[String] or RDD[Array[String]] in a future PR. I have sketched this out in the design doc (as well as handier versions of getTopics returning Strings). * Hyperparameters should be set differently for different inference/learning algorithms. See Asuncion et al. (2009) in the design doc for a good demonstration. I encourage good behavior via defaults and warning messages. Items planned for future PRs: * perplexity * API taking Strings * Should LDA be called LatentDirichletAllocation (and LDAModel be LatentDirichletAllocationModel)? * Pro: We may someday want LinearDiscriminantAnalysis. * Con: Very long names * Should LDA reside in clustering? Or do we want a sub-package? * mllib.topicmodel * mllib.clustering.topicmodel * Does the API seem reasonable and extensible? * Unit tests: * Should there be a test which checks a clustering results? E.g., train on a small, fake dataset with 2 very distinct topics/clusters, and ensure LDA finds those 2 topics/clusters. Does that sound useful or too flaky? This has not been tested much for scaling. I have run it on a laptop for 200 iterations on a 5MB dataset with 1000 terms and 5 topics. Running it for 500 iterations made it fail because of GC problems. I'm running larger scale tests & will put results here, but future PRs may need to improve the scaling. * dlwh for the initial implementation * + jegonzal for some code in the initial implementation * The many contributors towards topic model implementations in Spark which were referenced as a basis for this PR: akopich witgo yinxusen dlwh EntilZha jegonzal IlyaKozlov * Note: The plan is to include this full list in the authors if this PR gets merged. Please notify me if you prefer otherwise. CC: mengxr Authors: Joseph K. Bradley Joseph Gonzalez David Hall Guoqiang Li Xiangrui Meng Pedro Rodriguez Avanesov Valeriy Xusen Yin Closes #2388 Closes #4047 from jkbradley/davidhall-lda and squashes the following commits: 77e8814 [Joseph K. Bradley] small doc fix 5c74345 [Joseph K. Bradley] cleaned up doc based on code review 589728b [Joseph K. Bradley] Updates per code review. Main change was in LDAExample for faster vocab computation. Also updated PeriodicGraphCheckpointerSuite.scala to clean up checkpoint files at end e3980d2 [Joseph K. Bradley] cleaned up PeriodicGraphCheckpointerSuite.scala 74487e5 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into davidhall-lda 4ae2a7d [Joseph K. Bradley] removed duplicate graphx dependency in mllib/pom.xml e391474 [Joseph K. Bradley] Removed LDATiming. Added PeriodicGraphCheckpointerSuite.scala. Small LDA cleanups. e8d8acf [Joseph K. Bradley] Added catch for BreakIterator exception. Improved preprocessing to reduce passes over data 1a231b4 [Joseph K. Bradley] fixed scalastyle 91aadfe [Joseph K. Bradley] Added Java-friendly run method to LDA. Added Java test suite for LDA. Changed LDAModel.describeTopics to return Java-friendly type b75472d [Joseph K. Bradley] merged improvements from LDATiming into LDAExample. Will remove LDATiming after done testing 993ca56 [Joseph K. Bradley] * Removed Document type in favor of (Long, Vector) * Changed doc ID restriction to be: id must be nonnegative and unique in the doc (instead of 0,1,2,...) * Add checks for valid ranges of eta, alpha * Rename “LearningState” to “EMOptimizer” * Renamed params: termSmoothing -> topicConcentration, topicSmoothing -> docConcentration * Also added aliases alpha, beta cb5a319 [Joseph K. Bradley] Added checkpointing to LDA * new class PeriodicGraphCheckpointer * params checkpointDir, checkpointInterval to LDA 43c1c40 [Joseph K. Bradley] small cleanup 0b90393 [Joseph K. Bradley] renamed LDA LearningState.collectTopicTotals to globalTopicTotals 77a2c85 [Joseph K. Bradley] Moved auto term,topic smoothing computation to get*Smoothing methods. Changed word to term in some places. Updated LDAExample to use default smoothing amounts. fb1e7b5 [Xiangrui Meng] minor 08d59a3 [Xiangrui Meng] reset spacing 9fe0b95 [Xiangrui Meng] optimize aggregateMessages cec0a9c [Xiangrui Meng] * -> *= 6cb11b0 [Xiangrui Meng] optimize computePTopic 9eb3d02 [Xiangrui Meng] + -> += 892530c [Xiangrui Meng] use axpy 45cc7f2 [Xiangrui Meng] mapPart -> flatMap ce53be9 [Joseph K. Bradley] fixed example name 75749e7 [Joseph K. Bradley] scala style fix 9f2a492 [Joseph K. Bradley] Unit tests and fixes for LDA, now ready for PR 377ebd9 [Joseph K. Bradley] separated LDA models into own file. more cleanups before PR 2d40006 [Joseph K. Bradley] cleanups before PR 2891e89 [Joseph K. Bradley] Prepped LDA main class for PR, but some cleanups remain 0cb7187 [Joseph K. Bradley] Added 3 files from dlwh LDA implementation --- .../spark/examples/mllib/LDAExample.scala | 283 ++++++++++ .../apache/spark/mllib/clustering/LDA.scala | 519 ++++++++++++++++++ .../spark/mllib/clustering/LDAModel.scala | 351 ++++++++++++ .../impl/PeriodicGraphCheckpointer.scala | 179 ++++++ .../spark/mllib/clustering/JavaLDASuite.java | 119 ++++ .../spark/mllib/clustering/LDASuite.scala | 153 ++++++ .../impl/PeriodicGraphCheckpointerSuite.scala | 187 +++++++ 7 files changed, 1791 insertions(+) create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala create mode 100644 mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala 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 new file mode 100644 index 0000000000000..f4c545ad70e96 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -0,0 +1,283 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.text.BreakIterator + +import scala.collection.mutable + +import scopt.OptionParser + +import org.apache.log4j.{Level, Logger} + +import org.apache.spark.{SparkContext, SparkConf} +import org.apache.spark.mllib.clustering.LDA +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.rdd.RDD + + +/** + * An example Latent Dirichlet Allocation (LDA) app. Run with + * {{{ + * ./bin/run-example mllib.LDAExample [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object LDAExample { + + private case class Params( + input: Seq[String] = Seq.empty, + k: Int = 20, + maxIterations: Int = 10, + docConcentration: Double = -1, + topicConcentration: Double = -1, + vocabSize: Int = 10000, + stopwordFile: String = "", + checkpointDir: Option[String] = None, + checkpointInterval: Int = 10) extends AbstractParams[Params] + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("LDAExample") { + head("LDAExample: an example LDA app for plain text data.") + opt[Int]("k") + .text(s"number of topics. default: ${defaultParams.k}") + .action((x, c) => c.copy(k = x)) + opt[Int]("maxIterations") + .text(s"number of iterations of learning. default: ${defaultParams.maxIterations}") + .action((x, c) => c.copy(maxIterations = x)) + opt[Double]("docConcentration") + .text(s"amount of topic smoothing to use (> 1.0) (-1=auto)." + + s" default: ${defaultParams.docConcentration}") + .action((x, c) => c.copy(docConcentration = x)) + opt[Double]("topicConcentration") + .text(s"amount of term (word) smoothing to use (> 1.0) (-1=auto)." + + s" default: ${defaultParams.topicConcentration}") + .action((x, c) => c.copy(topicConcentration = x)) + opt[Int]("vocabSize") + .text(s"number of distinct word types to use, chosen by frequency. (-1=all)" + + s" default: ${defaultParams.vocabSize}") + .action((x, c) => c.copy(vocabSize = x)) + opt[String]("stopwordFile") + .text(s"filepath for a list of stopwords. Note: This must fit on a single machine." + + s" default: ${defaultParams.stopwordFile}") + .action((x, c) => c.copy(stopwordFile = x)) + opt[String]("checkpointDir") + .text(s"Directory for checkpointing intermediate results." + + s" Checkpointing helps with recovery and eliminates temporary shuffle files on disk." + + s" default: ${defaultParams.checkpointDir}") + .action((x, c) => c.copy(checkpointDir = Some(x))) + opt[Int]("checkpointInterval") + .text(s"Iterations between each checkpoint. Only used if checkpointDir is set." + + s" default: ${defaultParams.checkpointInterval}") + .action((x, c) => c.copy(checkpointInterval = x)) + arg[String]("...") + .text("input paths (directories) to plain text corpora." + + " Each text file line should hold 1 document.") + .unbounded() + .required() + .action((x, c) => c.copy(input = c.input :+ x)) + } + + parser.parse(args, defaultParams).map { params => + run(params) + }.getOrElse { + parser.showUsageAsError + sys.exit(1) + } + } + + private def run(params: Params) { + val conf = new SparkConf().setAppName(s"LDAExample with $params") + val sc = new SparkContext(conf) + + Logger.getRootLogger.setLevel(Level.WARN) + + // Load documents, and prepare them for LDA. + val preprocessStart = System.nanoTime() + val (corpus, vocabArray, actualNumTokens) = + preprocess(sc, params.input, params.vocabSize, params.stopwordFile) + corpus.cache() + val actualCorpusSize = corpus.count() + val actualVocabSize = vocabArray.size + val preprocessElapsed = (System.nanoTime() - preprocessStart) / 1e9 + + println() + println(s"Corpus summary:") + println(s"\t Training set size: $actualCorpusSize documents") + println(s"\t Vocabulary size: $actualVocabSize terms") + println(s"\t Training set size: $actualNumTokens tokens") + println(s"\t Preprocessing time: $preprocessElapsed sec") + println() + + // Run LDA. + val lda = new LDA() + lda.setK(params.k) + .setMaxIterations(params.maxIterations) + .setDocConcentration(params.docConcentration) + .setTopicConcentration(params.topicConcentration) + .setCheckpointInterval(params.checkpointInterval) + if (params.checkpointDir.nonEmpty) { + lda.setCheckpointDir(params.checkpointDir.get) + } + val startTime = System.nanoTime() + val ldaModel = lda.run(corpus) + val elapsed = (System.nanoTime() - startTime) / 1e9 + + println(s"Finished training LDA model. Summary:") + println(s"\t Training time: $elapsed sec") + val avgLogLikelihood = ldaModel.logLikelihood / actualCorpusSize.toDouble + println(s"\t Training data average log likelihood: $avgLogLikelihood") + println() + + // Print the topics, showing the top-weighted terms for each topic. + val topicIndices = ldaModel.describeTopics(maxTermsPerTopic = 10) + val topics = topicIndices.map { case (terms, termWeights) => + terms.zip(termWeights).map { case (term, weight) => (vocabArray(term.toInt), weight) } + } + println(s"${params.k} topics:") + topics.zipWithIndex.foreach { case (topic, i) => + println(s"TOPIC $i") + topic.foreach { case (term, weight) => + println(s"$term\t$weight") + } + println() + } + + } + + /** + * Load documents, tokenize them, create vocabulary, and prepare documents as term count vectors. + * @return (corpus, vocabulary as array, total token count in corpus) + */ + private def preprocess( + sc: SparkContext, + paths: Seq[String], + vocabSize: Int, + stopwordFile: String): (RDD[(Long, Vector)], Array[String], Long) = { + + // Get dataset of document texts + // One document per line in each text file. + val textRDD: RDD[String] = sc.textFile(paths.mkString(",")) + + // Split text into words + val tokenizer = new SimpleTokenizer(sc, stopwordFile) + val tokenized: RDD[(Long, IndexedSeq[String])] = textRDD.zipWithIndex().map { case (text, id) => + id -> tokenizer.getWords(text) + } + tokenized.cache() + + // Counts words: RDD[(word, wordCount)] + val wordCounts: RDD[(String, Long)] = tokenized + .flatMap { case (_, tokens) => tokens.map(_ -> 1L) } + .reduceByKey(_ + _) + wordCounts.cache() + val fullVocabSize = wordCounts.count() + // Select vocab + // (vocab: Map[word -> id], total tokens after selecting vocab) + val (vocab: Map[String, Int], selectedTokenCount: Long) = { + val tmpSortedWC: Array[(String, Long)] = if (vocabSize == -1 || fullVocabSize <= vocabSize) { + // Use all terms + wordCounts.collect().sortBy(-_._2) + } else { + // Sort terms to select vocab + wordCounts.sortBy(_._2, ascending = false).take(vocabSize) + } + (tmpSortedWC.map(_._1).zipWithIndex.toMap, tmpSortedWC.map(_._2).sum) + } + + val documents = tokenized.map { case (id, tokens) => + // Filter tokens by vocabulary, and create word count vector representation of document. + val wc = new mutable.HashMap[Int, Int]() + tokens.foreach { term => + if (vocab.contains(term)) { + val termIndex = vocab(term) + wc(termIndex) = wc.getOrElse(termIndex, 0) + 1 + } + } + val indices = wc.keys.toArray.sorted + val values = indices.map(i => wc(i).toDouble) + + val sb = Vectors.sparse(vocab.size, indices, values) + (id, sb) + } + + val vocabArray = new Array[String](vocab.size) + vocab.foreach { case (term, i) => vocabArray(i) = term } + + (documents, vocabArray, selectedTokenCount) + } +} + +/** + * Simple Tokenizer. + * + * TODO: Formalize the interface, and make this a public class in mllib.feature + */ +private class SimpleTokenizer(sc: SparkContext, stopwordFile: String) extends Serializable { + + private val stopwords: Set[String] = if (stopwordFile.isEmpty) { + Set.empty[String] + } else { + val stopwordText = sc.textFile(stopwordFile).collect() + stopwordText.flatMap(_.stripMargin.split("\\s+")).toSet + } + + // Matches sequences of Unicode letters + private val allWordRegex = "^(\\p{L}*)$".r + + // Ignore words shorter than this length. + private val minWordLength = 3 + + def getWords(text: String): IndexedSeq[String] = { + + val words = new mutable.ArrayBuffer[String]() + + // Use Java BreakIterator to tokenize text into words. + val wb = BreakIterator.getWordInstance + wb.setText(text) + + // current,end index start,end of each word + var current = wb.first() + var end = wb.next() + while (end != BreakIterator.DONE) { + // Convert to lowercase + val word: String = text.substring(current, end).toLowerCase + // Remove short words and strings that aren't only letters + word match { + case allWordRegex(w) if w.length >= minWordLength && !stopwords.contains(w) => + words += w + case _ => + } + + current = end + try { + end = wb.next() + } catch { + case e: Exception => + // Ignore remaining text in line. + // This is a known bug in BreakIterator (for some Java versions), + // which fails when it sees certain characters. + end = BreakIterator.DONE + } + } + words + } + +} 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 new file mode 100644 index 0000000000000..d8f82867a09d2 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDA.scala @@ -0,0 +1,519 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.clustering + +import java.util.Random + +import breeze.linalg.{DenseVector => BDV, normalize, axpy => brzAxpy} + +import org.apache.spark.Logging +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaPairRDD +import org.apache.spark.graphx._ +import org.apache.spark.graphx.impl.GraphImpl +import org.apache.spark.mllib.impl.PeriodicGraphCheckpointer +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + + +/** + * :: Experimental :: + * + * Latent Dirichlet Allocation (LDA), a topic model designed for text documents. + * + * Terminology: + * - "word" = "term": an element of the vocabulary + * - "token": instance of a term appearing in a document + * - "topic": multinomial distribution over words representing some concept + * + * Currently, the underlying implementation uses Expectation-Maximization (EM), implemented + * according to the Asuncion et al. (2009) paper referenced below. + * + * References: + * - Original LDA paper (journal version): + * Blei, Ng, and Jordan. "Latent Dirichlet Allocation." JMLR, 2003. + * - This class implements their "smoothed" LDA model. + * - Paper which clearly explains several algorithms, including EM: + * Asuncion, Welling, Smyth, and Teh. + * "On Smoothing and Inference for Topic Models." UAI, 2009. + */ +@Experimental +class LDA private ( + private var k: Int, + private var maxIterations: Int, + private var docConcentration: Double, + private var topicConcentration: Double, + private var seed: Long, + private var checkpointDir: Option[String], + private var checkpointInterval: Int) extends Logging { + + def this() = this(k = 10, maxIterations = 20, docConcentration = -1, topicConcentration = -1, + seed = Utils.random.nextLong(), checkpointDir = None, checkpointInterval = 10) + + /** + * Number of topics to infer. I.e., the number of soft cluster centers. + */ + def getK: Int = k + + /** + * Number of topics to infer. I.e., the number of soft cluster centers. + * (default = 10) + */ + def setK(k: Int): this.type = { + require(k > 0, s"LDA k (number of clusters) must be > 0, but was set to $k") + this.k = k + this + } + + /** + * Concentration parameter (commonly named "alpha") for the prior placed on documents' + * distributions over topics ("theta"). + * + * This is the parameter to a symmetric Dirichlet distribution. + */ + def getDocConcentration: Double = { + if (this.docConcentration == -1) { + (50.0 / k) + 1.0 + } else { + this.docConcentration + } + } + + /** + * Concentration parameter (commonly named "alpha") for the prior placed on documents' + * distributions over topics ("theta"). + * + * This is the parameter to a symmetric Dirichlet distribution. + * + * This value should be > 1.0, where larger values mean more smoothing (more regularization). + * If set to -1, then docConcentration is set automatically. + * (default = -1 = automatic) + * + * Automatic setting of parameter: + * - For EM: default = (50 / k) + 1. + * - The 50/k is common in LDA libraries. + * - The +1 follows Asuncion et al. (2009), who recommend a +1 adjustment for EM. + * + * Note: The restriction > 1.0 may be relaxed in the future (allowing sparse solutions), + * but values in (0,1) are not yet supported. + */ + def setDocConcentration(docConcentration: Double): this.type = { + require(docConcentration > 1.0 || docConcentration == -1.0, + s"LDA docConcentration must be > 1.0 (or -1 for auto), but was set to $docConcentration") + this.docConcentration = docConcentration + this + } + + /** Alias for [[getDocConcentration]] */ + def getAlpha: Double = getDocConcentration + + /** Alias for [[setDocConcentration()]] */ + def setAlpha(alpha: Double): this.type = setDocConcentration(alpha) + + /** + * Concentration parameter (commonly named "beta" or "eta") for the prior placed on topics' + * distributions over terms. + * + * This is the parameter to a symmetric Dirichlet distribution. + * + * Note: The topics' distributions over terms are called "beta" in the original LDA paper + * by Blei et al., but are called "phi" in many later papers such as Asuncion et al., 2009. + */ + def getTopicConcentration: Double = { + if (this.topicConcentration == -1) { + 1.1 + } else { + this.topicConcentration + } + } + + /** + * Concentration parameter (commonly named "beta" or "eta") for the prior placed on topics' + * distributions over terms. + * + * This is the parameter to a symmetric Dirichlet distribution. + * + * Note: The topics' distributions over terms are called "beta" in the original LDA paper + * by Blei et al., but are called "phi" in many later papers such as Asuncion et al., 2009. + * + * This value should be > 0.0. + * If set to -1, then topicConcentration is set automatically. + * (default = -1 = automatic) + * + * Automatic setting of parameter: + * - For EM: default = 0.1 + 1. + * - The 0.1 gives a small amount of smoothing. + * - The +1 follows Asuncion et al. (2009), who recommend a +1 adjustment for EM. + * + * Note: The restriction > 1.0 may be relaxed in the future (allowing sparse solutions), + * but values in (0,1) are not yet supported. + */ + def setTopicConcentration(topicConcentration: Double): this.type = { + require(topicConcentration > 1.0 || topicConcentration == -1.0, + s"LDA topicConcentration must be > 1.0 (or -1 for auto), but was set to $topicConcentration") + this.topicConcentration = topicConcentration + this + } + + /** Alias for [[getTopicConcentration]] */ + def getBeta: Double = getTopicConcentration + + /** Alias for [[setTopicConcentration()]] */ + def setBeta(beta: Double): this.type = setBeta(beta) + + /** + * Maximum number of iterations for learning. + */ + def getMaxIterations: Int = maxIterations + + /** + * Maximum number of iterations for learning. + * (default = 20) + */ + def setMaxIterations(maxIterations: Int): this.type = { + this.maxIterations = maxIterations + this + } + + /** Random seed */ + def getSeed: Long = seed + + /** Random seed */ + def setSeed(seed: Long): this.type = { + this.seed = seed + this + } + + /** + * Directory for storing checkpoint files during learning. + * This is not necessary, but checkpointing helps with recovery (when nodes fail). + * It also helps with eliminating temporary shuffle files on disk, which can be important when + * LDA is run for many iterations. + */ + def getCheckpointDir: Option[String] = checkpointDir + + /** + * Directory for storing checkpoint files during learning. + * This is not necessary, but checkpointing helps with recovery (when nodes fail). + * It also helps with eliminating temporary shuffle files on disk, which can be important when + * LDA is run for many iterations. + * + * NOTE: If the [[org.apache.spark.SparkContext.checkpointDir]] is already set, then the value + * given to LDA is ignored, and the existing directory is kept. + * + * (default = None) + */ + def setCheckpointDir(checkpointDir: String): this.type = { + this.checkpointDir = Some(checkpointDir) + this + } + + /** + * Clear the directory for storing checkpoint files during learning. + * If one is already set in the [[org.apache.spark.SparkContext]], then checkpointing will still + * occur; otherwise, no checkpointing will be used. + */ + def clearCheckpointDir(): this.type = { + this.checkpointDir = None + this + } + + /** + * Period (in iterations) between checkpoints. + * @see [[getCheckpointDir]] + */ + def getCheckpointInterval: Int = checkpointInterval + + /** + * Period (in iterations) between checkpoints. + * (default = 10) + * @see [[getCheckpointDir]] + */ + def setCheckpointInterval(checkpointInterval: Int): this.type = { + this.checkpointInterval = checkpointInterval + this + } + + /** + * Learn an LDA model using the given dataset. + * + * @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 + * (where the vocabulary size is the length of the vector). + * Document IDs must be unique and >= 0. + * @return Inferred LDA model + */ + def run(documents: RDD[(Long, Vector)]): DistributedLDAModel = { + val state = LDA.initialState(documents, k, getDocConcentration, getTopicConcentration, seed, + checkpointDir, checkpointInterval) + var iter = 0 + val iterationTimes = Array.fill[Double](maxIterations)(0) + while (iter < maxIterations) { + val start = System.nanoTime() + state.next() + val elapsedSeconds = (System.nanoTime() - start) / 1e9 + iterationTimes(iter) = elapsedSeconds + iter += 1 + } + state.graphCheckpointer.deleteAllCheckpoints() + new DistributedLDAModel(state, iterationTimes) + } + + /** Java-friendly version of [[run()]] */ + def run(documents: JavaPairRDD[java.lang.Long, Vector]): DistributedLDAModel = { + run(documents.rdd.asInstanceOf[RDD[(Long, Vector)]]) + } +} + + +private[clustering] object LDA { + + /* + DEVELOPERS NOTE: + + This implementation uses GraphX, where the graph is bipartite with 2 types of vertices: + - Document vertices + - indexed with unique indices >= 0 + - Store vectors of length k (# topics). + - Term vertices + - indexed {-1, -2, ..., -vocabSize} + - Store vectors of length k (# topics). + - Edges correspond to terms appearing in documents. + - Edges are directed Document -> Term. + - Edges are partitioned by documents. + + Info on EM implementation. + - We follow Section 2.2 from Asuncion et al., 2009. We use some of their notation. + - In this implementation, there is one edge for every unique term appearing in a document, + i.e., for every unique (document, term) pair. + - Notation: + - N_{wkj} = count of tokens of term w currently assigned to topic k in document j + - N_{*} where * is missing a subscript w/k/j is the count summed over missing subscript(s) + - gamma_{wjk} = P(z_i = k | x_i = w, d_i = j), + the probability of term x_i in document d_i having topic z_i. + - Data graph + - Document vertices store N_{kj} + - Term vertices store N_{wk} + - Edges store N_{wj}. + - Global data N_k + - Algorithm + - Initial state: + - Document and term vertices store random counts N_{wk}, N_{kj}. + - E-step: For each (document,term) pair i, compute P(z_i | x_i, d_i). + - Aggregate N_k from term vertices. + - Compute gamma_{wjk} for each possible topic k, from each triplet. + using inputs N_{wk}, N_{kj}, N_k. + - M-step: Compute sufficient statistics for hidden parameters phi and theta + (counts N_{wk}, N_{kj}, N_k). + - Document update: + - N_{kj} <- sum_w N_{wj} gamma_{wjk} + - N_j <- sum_k N_{kj} (only needed to output predictions) + - Term update: + - N_{wk} <- sum_j N_{wj} gamma_{wjk} + - N_k <- sum_w N_{wk} + + TODO: Add simplex constraints to allow alpha in (0,1). + See: Vorontsov and Potapenko. "Tutorial on Probabilistic Topic Modeling : Additive + Regularization for Stochastic Matrix Factorization." 2014. + */ + + /** + * Vector over topics (length k) of token counts. + * The meaning of these counts can vary, and it may or may not be normalized to be a distribution. + */ + type TopicCounts = BDV[Double] + + type TokenCount = Double + + /** Term vertex IDs are {-1, -2, ..., -vocabSize} */ + def term2index(term: Int): Long = -(1 + term.toLong) + + def index2term(termIndex: Long): Int = -(1 + termIndex).toInt + + def isDocumentVertex(v: (VertexId, _)): Boolean = v._1 >= 0 + + def isTermVertex(v: (VertexId, _)): Boolean = v._1 < 0 + + /** + * Optimizer for EM algorithm which stores data + parameter graph, plus algorithm parameters. + * + * @param graph EM graph, storing current parameter estimates in vertex descriptors and + * data (token counts) in edge descriptors. + * @param k Number of topics + * @param vocabSize Number of unique terms + * @param docConcentration "alpha" + * @param topicConcentration "beta" or "eta" + */ + class EMOptimizer( + var graph: Graph[TopicCounts, TokenCount], + val k: Int, + val vocabSize: Int, + val docConcentration: Double, + val topicConcentration: Double, + checkpointDir: Option[String], + checkpointInterval: Int) { + + private[LDA] val graphCheckpointer = new PeriodicGraphCheckpointer[TopicCounts, TokenCount]( + graph, checkpointDir, checkpointInterval) + + def next(): EMOptimizer = { + val eta = topicConcentration + val W = vocabSize + val alpha = docConcentration + + val N_k = globalTopicTotals + val sendMsg: EdgeContext[TopicCounts, TokenCount, (Boolean, TopicCounts)] => Unit = + (edgeContext) => { + // Compute N_{wj} gamma_{wjk} + val N_wj = edgeContext.attr + // E-STEP: Compute gamma_{wjk} (smoothed topic distributions), scaled by token count + // N_{wj}. + val scaledTopicDistribution: TopicCounts = + computePTopic(edgeContext.srcAttr, edgeContext.dstAttr, N_k, W, eta, alpha) *= N_wj + edgeContext.sendToDst((false, scaledTopicDistribution)) + edgeContext.sendToSrc((false, scaledTopicDistribution)) + } + // This is a hack to detect whether we could modify the values in-place. + // TODO: Add zero/seqOp/combOp option to aggregateMessages. (SPARK-5438) + val mergeMsg: ((Boolean, TopicCounts), (Boolean, TopicCounts)) => (Boolean, TopicCounts) = + (m0, m1) => { + val sum = + if (m0._1) { + m0._2 += m1._2 + } else if (m1._1) { + m1._2 += m0._2 + } else { + m0._2 + m1._2 + } + (true, sum) + } + // M-STEP: Aggregation computes new N_{kj}, N_{wk} counts. + val docTopicDistributions: VertexRDD[TopicCounts] = + graph.aggregateMessages[(Boolean, TopicCounts)](sendMsg, mergeMsg) + .mapValues(_._2) + // Update the vertex descriptors with the new counts. + val newGraph = GraphImpl.fromExistingRDDs(docTopicDistributions, graph.edges) + graph = newGraph + graphCheckpointer.updateGraph(newGraph) + globalTopicTotals = computeGlobalTopicTotals() + this + } + + /** + * Aggregate distributions over topics from all term vertices. + * + * Note: This executes an action on the graph RDDs. + */ + var globalTopicTotals: TopicCounts = computeGlobalTopicTotals() + + private def computeGlobalTopicTotals(): TopicCounts = { + val numTopics = k + graph.vertices.filter(isTermVertex).values.fold(BDV.zeros[Double](numTopics))(_ += _) + } + + } + + /** + * Compute gamma_{wjk}, a distribution over topics k. + */ + private def computePTopic( + docTopicCounts: TopicCounts, + termTopicCounts: TopicCounts, + totalTopicCounts: TopicCounts, + vocabSize: Int, + eta: Double, + alpha: Double): TopicCounts = { + val K = docTopicCounts.length + val N_j = docTopicCounts.data + val N_w = termTopicCounts.data + val N = totalTopicCounts.data + val eta1 = eta - 1.0 + val alpha1 = alpha - 1.0 + val Weta1 = vocabSize * eta1 + var sum = 0.0 + val gamma_wj = new Array[Double](K) + var k = 0 + while (k < K) { + val gamma_wjk = (N_w(k) + eta1) * (N_j(k) + alpha1) / (N(k) + Weta1) + gamma_wj(k) = gamma_wjk + sum += gamma_wjk + k += 1 + } + // normalize + BDV(gamma_wj) /= sum + } + + /** + * Compute bipartite term/doc graph. + */ + private def initialState( + docs: RDD[(Long, Vector)], + k: Int, + docConcentration: Double, + topicConcentration: Double, + randomSeed: Long, + checkpointDir: Option[String], + checkpointInterval: Int): EMOptimizer = { + // For each document, create an edge (Document -> Term) for each unique term in the document. + val edges: RDD[Edge[TokenCount]] = docs.flatMap { case (docID: Long, termCounts: Vector) => + // Add edges for terms with non-zero counts. + termCounts.toBreeze.activeIterator.filter(_._2 != 0.0).map { case (term, cnt) => + Edge(docID, term2index(term), cnt) + } + } + + val vocabSize = docs.take(1).head._2.size + + // 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(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 + } + ) + } + val docVertices = createVertices(_.srcId) + val termVertices = createVertices(_.dstId) + + // Partition such that edges are grouped by document + val graph = Graph(docVertices ++ termVertices, edges) + .partitionBy(PartitionStrategy.EdgePartition1D) + + new EMOptimizer(graph, k, vocabSize, docConcentration, topicConcentration, checkpointDir, + checkpointInterval) + } + +} 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 new file mode 100644 index 0000000000000..19e8aab6eabd7 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -0,0 +1,351 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.clustering + +import breeze.linalg.{DenseMatrix => BDM, normalize, sum => brzSum} + +import org.apache.spark.annotation.Experimental +import org.apache.spark.graphx.{VertexId, EdgeContext, Graph} +import org.apache.spark.mllib.linalg.{Vectors, Vector, Matrices, Matrix} +import org.apache.spark.rdd.RDD +import org.apache.spark.util.BoundedPriorityQueue + +/** + * :: Experimental :: + * + * Latent Dirichlet Allocation (LDA) model. + * + * This abstraction permits for different underlying representations, + * including local and distributed data structures. + */ +@Experimental +abstract class LDAModel private[clustering] { + + /** Number of topics */ + def k: Int + + /** Vocabulary size (number of terms or terms in the vocabulary) */ + def vocabSize: Int + + /** + * Inferred topics, where each topic is represented by a distribution over terms. + * This is a matrix of size vocabSize x k, where each column is a topic. + * No guarantees are given about the ordering of the topics. + */ + def topicsMatrix: Matrix + + /** + * Return the topics described by weighted terms. + * + * This limits the number of terms per topic. + * This is approximate; it may not return exactly the top-weighted terms for each topic. + * To get a more precise set of top terms, increase maxTermsPerTopic. + * + * @param maxTermsPerTopic Maximum number of terms to collect for each topic. + * @return Array over topics. Each topic is represented as a pair of matching arrays: + * (term indices, term weights in topic). + * Each topic's terms are sorted in order of decreasing weight. + */ + def describeTopics(maxTermsPerTopic: Int): Array[(Array[Int], Array[Double])] + + /** + * Return the topics described by weighted terms. + * + * WARNING: If vocabSize and k are large, this can return a large object! + * + * @return Array over topics. Each topic is represented as a pair of matching arrays: + * (term indices, term weights in topic). + * Each topic's terms are sorted in order of decreasing weight. + */ + def describeTopics(): Array[(Array[Int], Array[Double])] = describeTopics(vocabSize) + + /* TODO (once LDA can be trained with Strings or given a dictionary) + * Return the topics described by weighted terms. + * + * This is similar to [[describeTopics()]] but returns String values for terms. + * If this model was trained using Strings or was given a dictionary, then this method returns + * terms as text. Otherwise, this method returns terms as term indices. + * + * This limits the number of terms per topic. + * This is approximate; it may not return exactly the top-weighted terms for each topic. + * To get a more precise set of top terms, increase maxTermsPerTopic. + * + * @param maxTermsPerTopic Maximum number of terms to collect for each topic. + * @return Array over topics. Each topic is represented as a pair of matching arrays: + * (terms, term weights in topic) where terms are either the actual term text + * (if available) or the term indices. + * Each topic's terms are sorted in order of decreasing weight. + */ + // def describeTopicsAsStrings(maxTermsPerTopic: Int): Array[(Array[Double], Array[String])] + + /* TODO (once LDA can be trained with Strings or given a dictionary) + * Return the topics described by weighted terms. + * + * This is similar to [[describeTopics()]] but returns String values for terms. + * If this model was trained using Strings or was given a dictionary, then this method returns + * terms as text. Otherwise, this method returns terms as term indices. + * + * WARNING: If vocabSize and k are large, this can return a large object! + * + * @return Array over topics. Each topic is represented as a pair of matching arrays: + * (terms, term weights in topic) where terms are either the actual term text + * (if available) or the term indices. + * Each topic's terms are sorted in order of decreasing weight. + */ + // def describeTopicsAsStrings(): Array[(Array[Double], Array[String])] = + // describeTopicsAsStrings(vocabSize) + + /* TODO + * Compute the log likelihood of the observed tokens, given the current parameter estimates: + * log P(docs | topics, topic distributions for docs, alpha, eta) + * + * Note: + * - This excludes the prior. + * - Even with the prior, this is NOT the same as the data log likelihood given the + * hyperparameters. + * + * @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 + * (where the vocabulary size is the length of the vector). + * This must use the same vocabulary (ordering of term counts) as in training. + * Document IDs must be unique and >= 0. + * @return Estimated log likelihood of the data under this model + */ + // def logLikelihood(documents: RDD[(Long, Vector)]): Double + + /* TODO + * Compute the estimated topic distribution for each document. + * 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 + * (where the vocabulary size is the length of the vector). + * This must use the same vocabulary (ordering of term counts) as in training. + * Document IDs must be unique and >= 0. + * @return Estimated topic distribution for each document. + * The returned RDD may be zipped with the given RDD, where each returned vector + * is a multinomial distribution over topics. + */ + // def topicDistributions(documents: RDD[(Long, Vector)]): RDD[(Long, Vector)] + +} + +/** + * :: Experimental :: + * + * Local LDA model. + * This model stores only the inferred topics. + * It may be used for computing topics for new documents, but it may give less accurate answers + * than the [[DistributedLDAModel]]. + * + * @param topics Inferred topics (vocabSize x k matrix). + */ +@Experimental +class LocalLDAModel private[clustering] ( + private val topics: Matrix) extends LDAModel with Serializable { + + override def k: Int = topics.numCols + + override def vocabSize: Int = topics.numRows + + override def topicsMatrix: Matrix = topics + + override def describeTopics(maxTermsPerTopic: Int): Array[(Array[Int], Array[Double])] = { + val brzTopics = topics.toBreeze.toDenseMatrix + Range(0, k).map { topicIndex => + val topic = normalize(brzTopics(::, topicIndex), 1.0) + val (termWeights, terms) = + topic.toArray.zipWithIndex.sortBy(-_._1).take(maxTermsPerTopic).unzip + (terms.toArray, termWeights.toArray) + }.toArray + } + + // TODO + // override def logLikelihood(documents: RDD[(Long, Vector)]): Double = ??? + + // TODO: + // override def topicDistributions(documents: RDD[(Long, Vector)]): RDD[(Long, Vector)] = ??? + +} + +/** + * :: Experimental :: + * + * Distributed LDA model. + * This model stores the inferred topics, the full training dataset, and the topic distributions. + * When computing topics for new documents, it may give more accurate answers + * than the [[LocalLDAModel]]. + */ +@Experimental +class DistributedLDAModel private ( + private val graph: Graph[LDA.TopicCounts, LDA.TokenCount], + private val globalTopicTotals: LDA.TopicCounts, + val k: Int, + val vocabSize: Int, + private val docConcentration: Double, + private val topicConcentration: Double, + private[spark] val iterationTimes: Array[Double]) extends LDAModel { + + import LDA._ + + private[clustering] def this(state: LDA.EMOptimizer, iterationTimes: Array[Double]) = { + this(state.graph, state.globalTopicTotals, state.k, state.vocabSize, state.docConcentration, + state.topicConcentration, iterationTimes) + } + + /** + * Convert model to a local model. + * The local model stores the inferred topics but not the topic distributions for training + * documents. + */ + def toLocal: LocalLDAModel = new LocalLDAModel(topicsMatrix) + + /** + * Inferred topics, where each topic is represented by a distribution over terms. + * This is a matrix of size vocabSize x k, where each column is a topic. + * No guarantees are given about the ordering of the topics. + * + * WARNING: This matrix is collected from an RDD. Beware memory usage when vocabSize, k are large. + */ + override lazy val topicsMatrix: Matrix = { + // Collect row-major topics + val termTopicCounts: Array[(Int, TopicCounts)] = + graph.vertices.filter(_._1 < 0).map { case (termIndex, cnts) => + (index2term(termIndex), cnts) + }.collect() + // Convert to Matrix + val brzTopics = BDM.zeros[Double](vocabSize, k) + termTopicCounts.foreach { case (term, cnts) => + var j = 0 + while (j < k) { + brzTopics(term, j) = cnts(j) + j += 1 + } + } + Matrices.fromBreeze(brzTopics) + } + + override def describeTopics(maxTermsPerTopic: Int): Array[(Array[Int], Array[Double])] = { + val numTopics = k + // Note: N_k is not needed to find the top terms, but it is needed to normalize weights + // to a distribution over terms. + val N_k: TopicCounts = globalTopicTotals + val topicsInQueues: Array[BoundedPriorityQueue[(Double, Int)]] = + graph.vertices.filter(isTermVertex) + .mapPartitions { termVertices => + // For this partition, collect the most common terms for each topic in queues: + // queues(topic) = queue of (term weight, term index). + // Term weights are N_{wk} / N_k. + val queues = + Array.fill(numTopics)(new BoundedPriorityQueue[(Double, Int)](maxTermsPerTopic)) + for ((termId, n_wk) <- termVertices) { + var topic = 0 + while (topic < numTopics) { + queues(topic) += (n_wk(topic) / N_k(topic) -> index2term(termId.toInt)) + topic += 1 + } + } + Iterator(queues) + }.reduce { (q1, q2) => + q1.zip(q2).foreach { case (a, b) => a ++= b} + q1 + } + topicsInQueues.map { q => + val (termWeights, terms) = q.toArray.sortBy(-_._1).unzip + (terms.toArray, termWeights.toArray) + } + } + + // TODO + // override def logLikelihood(documents: RDD[(Long, Vector)]): Double = ??? + + /** + * Log likelihood of the observed tokens in the training set, + * given the current parameter estimates: + * log P(docs | topics, topic distributions for docs, alpha, eta) + * + * Note: + * - This excludes the prior; for that, use [[logPrior]]. + * - Even with [[logPrior]], this is NOT the same as the data log likelihood given the + * hyperparameters. + */ + lazy val logLikelihood: Double = { + val eta = topicConcentration + val alpha = docConcentration + assert(eta > 1.0) + assert(alpha > 1.0) + val N_k = globalTopicTotals + val smoothed_N_k: TopicCounts = N_k + (vocabSize * (eta - 1.0)) + // Edges: Compute token log probability from phi_{wk}, theta_{kj}. + val sendMsg: EdgeContext[TopicCounts, TokenCount, Double] => Unit = (edgeContext) => { + val N_wj = edgeContext.attr + val smoothed_N_wk: TopicCounts = edgeContext.dstAttr + (eta - 1.0) + val smoothed_N_kj: TopicCounts = edgeContext.srcAttr + (alpha - 1.0) + val phi_wk: TopicCounts = smoothed_N_wk :/ smoothed_N_k + val theta_kj: TopicCounts = normalize(smoothed_N_kj, 1.0) + val tokenLogLikelihood = N_wj * math.log(phi_wk.dot(theta_kj)) + edgeContext.sendToDst(tokenLogLikelihood) + } + graph.aggregateMessages[Double](sendMsg, _ + _) + .map(_._2).fold(0.0)(_ + _) + } + + /** + * Log probability of the current parameter estimate: + * log P(topics, topic distributions for docs | alpha, eta) + */ + lazy val logPrior: Double = { + val eta = topicConcentration + val alpha = docConcentration + // Term vertices: Compute phi_{wk}. Use to compute prior log probability. + // Doc vertex: Compute theta_{kj}. Use to compute prior log probability. + val N_k = globalTopicTotals + val smoothed_N_k: TopicCounts = N_k + (vocabSize * (eta - 1.0)) + val seqOp: (Double, (VertexId, TopicCounts)) => Double = { + case (sumPrior: Double, vertex: (VertexId, TopicCounts)) => + if (isTermVertex(vertex)) { + val N_wk = vertex._2 + val smoothed_N_wk: TopicCounts = N_wk + (eta - 1.0) + val phi_wk: TopicCounts = smoothed_N_wk :/ smoothed_N_k + (eta - 1.0) * brzSum(phi_wk.map(math.log)) + } else { + val N_kj = vertex._2 + val smoothed_N_kj: TopicCounts = N_kj + (alpha - 1.0) + val theta_kj: TopicCounts = normalize(smoothed_N_kj, 1.0) + (alpha - 1.0) * brzSum(theta_kj.map(math.log)) + } + } + graph.vertices.aggregate(0.0)(seqOp, _ + _) + } + + /** + * For each document in the training set, return the distribution over topics for that document + * (i.e., "theta_doc"). + * + * @return RDD of (document ID, topic distribution) pairs + */ + def topicDistributions: RDD[(Long, Vector)] = { + graph.vertices.filter(LDA.isDocumentVertex).map { case (docID, topicCounts) => + (docID.toLong, Vectors.fromBreeze(normalize(topicCounts, 1.0))) + } + } + + // TODO: + // override def topicDistributions(documents: RDD[(Long, Vector)]): RDD[(Long, Vector)] = ??? + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala new file mode 100644 index 0000000000000..76672fe51e834 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.impl + +import scala.collection.mutable + +import org.apache.hadoop.fs.{Path, FileSystem} + +import org.apache.spark.Logging +import org.apache.spark.graphx.Graph +import org.apache.spark.storage.StorageLevel + + +/** + * This class helps with persisting and checkpointing Graphs. + * Specifically, it automatically handles persisting and (optionally) checkpointing, as well as + * unpersisting and removing checkpoint files. + * + * Users should call [[PeriodicGraphCheckpointer.updateGraph()]] when a new graph has been created, + * before the graph has been materialized. After updating [[PeriodicGraphCheckpointer]], users are + * responsible for materializing the graph to ensure that persisting and checkpointing actually + * occur. + * + * When [[PeriodicGraphCheckpointer.updateGraph()]] is called, this does the following: + * - Persist new graph (if not yet persisted), and put in queue of persisted graphs. + * - Unpersist graphs from queue until there are at most 3 persisted graphs. + * - If using checkpointing and the checkpoint interval has been reached, + * - Checkpoint the new graph, and put in a queue of checkpointed graphs. + * - Remove older checkpoints. + * + * WARNINGS: + * - This class should NOT be copied (since copies may conflict on which Graphs should be + * checkpointed). + * - This class removes checkpoint files once later graphs have been checkpointed. + * However, references to the older graphs will still return isCheckpointed = true. + * + * Example usage: + * {{{ + * val (graph1, graph2, graph3, ...) = ... + * val cp = new PeriodicGraphCheckpointer(graph1, dir, 2) + * graph1.vertices.count(); graph1.edges.count() + * // persisted: graph1 + * cp.updateGraph(graph2) + * graph2.vertices.count(); graph2.edges.count() + * // persisted: graph1, graph2 + * // checkpointed: graph2 + * cp.updateGraph(graph3) + * graph3.vertices.count(); graph3.edges.count() + * // persisted: graph1, graph2, graph3 + * // checkpointed: graph2 + * cp.updateGraph(graph4) + * graph4.vertices.count(); graph4.edges.count() + * // persisted: graph2, graph3, graph4 + * // checkpointed: graph4 + * cp.updateGraph(graph5) + * graph5.vertices.count(); graph5.edges.count() + * // persisted: graph3, graph4, graph5 + * // checkpointed: graph4 + * }}} + * + * @param currentGraph Initial graph + * @param checkpointDir The directory for storing checkpoint files + * @param checkpointInterval Graphs will be checkpointed at this interval + * @tparam VD Vertex descriptor type + * @tparam ED Edge descriptor type + * + * TODO: Generalize this for Graphs and RDDs, and move it out of MLlib. + */ +private[mllib] class PeriodicGraphCheckpointer[VD, ED]( + var currentGraph: Graph[VD, ED], + val checkpointDir: Option[String], + val checkpointInterval: Int) extends Logging { + + /** FIFO queue of past checkpointed RDDs */ + private val checkpointQueue = mutable.Queue[Graph[VD, ED]]() + + /** FIFO queue of past persisted RDDs */ + private val persistedQueue = mutable.Queue[Graph[VD, ED]]() + + /** Number of times [[updateGraph()]] has been called */ + private var updateCount = 0 + + /** + * Spark Context for the Graphs given to this checkpointer. + * NOTE: This code assumes that only one SparkContext is used for the given graphs. + */ + private val sc = currentGraph.vertices.sparkContext + + // If a checkpoint directory is given, and there's no prior checkpoint directory, + // then set the checkpoint directory with the given one. + if (checkpointDir.nonEmpty && sc.getCheckpointDir.isEmpty) { + sc.setCheckpointDir(checkpointDir.get) + } + + updateGraph(currentGraph) + + /** + * Update [[currentGraph]] with a new graph. Handle persistence and checkpointing as needed. + * Since this handles persistence and checkpointing, this should be called before the graph + * has been materialized. + * + * @param newGraph New graph created from previous graphs in the lineage. + */ + def updateGraph(newGraph: Graph[VD, ED]): Unit = { + if (newGraph.vertices.getStorageLevel == StorageLevel.NONE) { + newGraph.persist() + } + persistedQueue.enqueue(newGraph) + // We try to maintain 2 Graphs in persistedQueue to support the semantics of this class: + // Users should call [[updateGraph()]] when a new graph has been created, + // before the graph has been materialized. + while (persistedQueue.size > 3) { + val graphToUnpersist = persistedQueue.dequeue() + graphToUnpersist.unpersist(blocking = false) + } + updateCount += 1 + + // Handle checkpointing (after persisting) + if ((updateCount % checkpointInterval) == 0 && sc.getCheckpointDir.nonEmpty) { + // Add new checkpoint before removing old checkpoints. + newGraph.checkpoint() + checkpointQueue.enqueue(newGraph) + // Remove checkpoints before the latest one. + var canDelete = true + while (checkpointQueue.size > 1 && canDelete) { + // Delete the oldest checkpoint only if the next checkpoint exists. + if (checkpointQueue.get(1).get.isCheckpointed) { + removeCheckpointFile() + } else { + canDelete = false + } + } + } + } + + /** + * Call this at the end to delete any remaining checkpoint files. + */ + def deleteAllCheckpoints(): Unit = { + while (checkpointQueue.size > 0) { + removeCheckpointFile() + } + } + + /** + * Dequeue the oldest checkpointed Graph, and remove its checkpoint files. + * This prints a warning but does not fail if the files cannot be removed. + */ + private def removeCheckpointFile(): Unit = { + val old = checkpointQueue.dequeue() + // Since the old checkpoint is not deleted by Spark, we manually delete it. + val fs = FileSystem.get(sc.hadoopConfiguration) + old.getCheckpointFiles.foreach { checkpointFile => + try { + fs.delete(new Path(checkpointFile), true) + } catch { + case e: Exception => + logWarning("PeriodicGraphCheckpointer could not remove old checkpoint file: " + + checkpointFile) + } + } + } + +} diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java new file mode 100644 index 0000000000000..dc10aa67c7c1f --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java @@ -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.mllib.clustering; + +import java.io.Serializable; +import java.util.ArrayList; + +import org.apache.spark.api.java.JavaRDD; +import scala.Tuple2; + +import org.junit.After; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertArrayEquals; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.Vector; + + +public class JavaLDASuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaLDA"); + ArrayList> tinyCorpus = new ArrayList>(); + for (int i = 0; i < LDASuite$.MODULE$.tinyCorpus().length; i++) { + tinyCorpus.add(new Tuple2((Long)LDASuite$.MODULE$.tinyCorpus()[i]._1(), + LDASuite$.MODULE$.tinyCorpus()[i]._2())); + } + JavaRDD> tmpCorpus = sc.parallelize(tinyCorpus, 2); + corpus = JavaPairRDD.fromJavaRDD(tmpCorpus); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + public void localLDAModel() { + LocalLDAModel model = new LocalLDAModel(LDASuite$.MODULE$.tinyTopics()); + + // Check: basic parameters + assertEquals(model.k(), tinyK); + assertEquals(model.vocabSize(), tinyVocabSize); + assertEquals(model.topicsMatrix(), tinyTopics); + + // Check: describeTopics() with all terms + Tuple2[] fullTopicSummary = model.describeTopics(); + assertEquals(fullTopicSummary.length, tinyK); + for (int i = 0; i < fullTopicSummary.length; i++) { + assertArrayEquals(fullTopicSummary[i]._1(), tinyTopicDescription[i]._1()); + assertArrayEquals(fullTopicSummary[i]._2(), tinyTopicDescription[i]._2(), 1e-5); + } + } + + @Test + public void distributedLDAModel() { + int k = 3; + double topicSmoothing = 1.2; + double termSmoothing = 1.2; + + // Train a model + LDA lda = new LDA(); + lda.setK(k) + .setDocConcentration(topicSmoothing) + .setTopicConcentration(termSmoothing) + .setMaxIterations(5) + .setSeed(12345); + + DistributedLDAModel model = lda.run(corpus); + + // Check: basic parameters + LocalLDAModel localModel = model.toLocal(); + assertEquals(model.k(), k); + assertEquals(localModel.k(), k); + assertEquals(model.vocabSize(), tinyVocabSize); + assertEquals(localModel.vocabSize(), tinyVocabSize); + assertEquals(model.topicsMatrix(), localModel.topicsMatrix()); + + // Check: topic summaries + Tuple2[] roundedTopicSummary = model.describeTopics(); + assertEquals(roundedTopicSummary.length, k); + Tuple2[] roundedLocalTopicSummary = localModel.describeTopics(); + assertEquals(roundedLocalTopicSummary.length, k); + + // Check: log probabilities + assert(model.logLikelihood() < 0.0); + assert(model.logPrior() < 0.0); + } + + private static int tinyK = LDASuite$.MODULE$.tinyK(); + private static int tinyVocabSize = LDASuite$.MODULE$.tinyVocabSize(); + private static Matrix tinyTopics = LDASuite$.MODULE$.tinyTopics(); + private static Tuple2[] tinyTopicDescription = + LDASuite$.MODULE$.tinyTopicDescription(); + JavaPairRDD corpus; + +} 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 new file mode 100644 index 0000000000000..302d751eb8a94 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.clustering + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.{DenseMatrix, Matrix, Vectors} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ + +class LDASuite extends FunSuite with MLlibTestSparkContext { + + import LDASuite._ + + test("LocalLDAModel") { + val model = new LocalLDAModel(tinyTopics) + + // Check: basic parameters + assert(model.k === tinyK) + assert(model.vocabSize === tinyVocabSize) + assert(model.topicsMatrix === tinyTopics) + + // Check: describeTopics() with all terms + val fullTopicSummary = model.describeTopics() + assert(fullTopicSummary.size === tinyK) + fullTopicSummary.zip(tinyTopicDescription).foreach { + case ((algTerms, algTermWeights), (terms, termWeights)) => + assert(algTerms === terms) + assert(algTermWeights === termWeights) + } + + // Check: describeTopics() with some terms + val smallNumTerms = 3 + val smallTopicSummary = model.describeTopics(maxTermsPerTopic = smallNumTerms) + smallTopicSummary.zip(tinyTopicDescription).foreach { + case ((algTerms, algTermWeights), (terms, termWeights)) => + assert(algTerms === terms.slice(0, smallNumTerms)) + assert(algTermWeights === termWeights.slice(0, smallNumTerms)) + } + } + + test("running and DistributedLDAModel") { + val k = 3 + val topicSmoothing = 1.2 + val termSmoothing = 1.2 + + // Train a model + val lda = new LDA() + lda.setK(k) + .setDocConcentration(topicSmoothing) + .setTopicConcentration(termSmoothing) + .setMaxIterations(5) + .setSeed(12345) + val corpus = sc.parallelize(tinyCorpus, 2) + + val model: DistributedLDAModel = lda.run(corpus) + + // Check: basic parameters + val localModel = model.toLocal + assert(model.k === k) + assert(localModel.k === k) + assert(model.vocabSize === tinyVocabSize) + assert(localModel.vocabSize === tinyVocabSize) + assert(model.topicsMatrix === localModel.topicsMatrix) + + // Check: topic summaries + // The odd decimal formatting and sorting is a hack to do a robust comparison. + val roundedTopicSummary = model.describeTopics().map { case (terms, termWeights) => + // cut values to 3 digits after the decimal place + terms.zip(termWeights).map { case (term, weight) => + ("%.3f".format(weight).toDouble, term.toInt) + } + }.sortBy(_.mkString("")) + val roundedLocalTopicSummary = localModel.describeTopics().map { case (terms, termWeights) => + // cut values to 3 digits after the decimal place + terms.zip(termWeights).map { case (term, weight) => + ("%.3f".format(weight).toDouble, term.toInt) + } + }.sortBy(_.mkString("")) + roundedTopicSummary.zip(roundedLocalTopicSummary).foreach { case (t1, t2) => + assert(t1 === t2) + } + + // Check: per-doc topic distributions + val topicDistributions = model.topicDistributions.collect() + // Ensure all documents are covered. + assert(topicDistributions.size === tinyCorpus.size) + assert(tinyCorpus.map(_._1).toSet === topicDistributions.map(_._1).toSet) + // Ensure we have proper distributions + topicDistributions.foreach { case (docId, topicDistribution) => + assert(topicDistribution.size === tinyK) + assert(topicDistribution.toArray.sum ~== 1.0 absTol 1e-5) + } + + // Check: log probabilities + assert(model.logLikelihood < 0.0) + assert(model.logPrior < 0.0) + } + + test("vertex indexing") { + // Check vertex ID indexing and conversions. + val docIds = Array(0, 1, 2) + val docVertexIds = docIds + val termIds = Array(0, 1, 2) + val termVertexIds = Array(-1, -2, -3) + assert(docVertexIds.forall(i => !LDA.isTermVertex((i.toLong, 0)))) + assert(termIds.map(LDA.term2index) === termVertexIds) + assert(termVertexIds.map(i => LDA.index2term(i.toLong)) === termIds) + assert(termVertexIds.forall(i => LDA.isTermVertex((i.toLong, 0)))) + } +} + +private[clustering] object LDASuite { + + def tinyK: Int = 3 + def tinyVocabSize: Int = 5 + def tinyTopicsAsArray: Array[Array[Double]] = Array( + Array[Double](0.1, 0.2, 0.3, 0.4, 0.0), // topic 0 + Array[Double](0.5, 0.05, 0.05, 0.1, 0.3), // topic 1 + Array[Double](0.2, 0.2, 0.05, 0.05, 0.5) // topic 2 + ) + def tinyTopics: Matrix = new DenseMatrix(numRows = tinyVocabSize, numCols = tinyK, + values = tinyTopicsAsArray.fold(Array.empty[Double])(_ ++ _)) + def tinyTopicDescription: Array[(Array[Int], Array[Double])] = tinyTopicsAsArray.map { topic => + val (termWeights, terms) = topic.zipWithIndex.sortBy(-_._1).unzip + (terms.toArray, termWeights.toArray) + } + + def tinyCorpus = Array( + Vectors.dense(1, 3, 0, 2, 8), + Vectors.dense(0, 2, 1, 0, 4), + Vectors.dense(2, 3, 12, 3, 1), + Vectors.dense(0, 3, 1, 9, 8), + Vectors.dense(1, 1, 4, 2, 6) + ).zipWithIndex.map { case (wordCounts, docId) => (docId.toLong, wordCounts) } + assert(tinyCorpus.forall(_._2.size == tinyVocabSize)) // sanity check for test data + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala new file mode 100644 index 0000000000000..dac28a369b5b2 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala @@ -0,0 +1,187 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.impl + +import org.scalatest.FunSuite + +import org.apache.hadoop.fs.{FileSystem, Path} + +import org.apache.spark.SparkContext +import org.apache.spark.graphx.{Edge, Graph} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils + + +class PeriodicGraphCheckpointerSuite extends FunSuite with MLlibTestSparkContext { + + import PeriodicGraphCheckpointerSuite._ + + // TODO: Do I need to call count() on the graphs' RDDs? + + test("Persisting") { + var graphsToCheck = Seq.empty[GraphToCheck] + + val graph1 = createGraph(sc) + val checkpointer = new PeriodicGraphCheckpointer(graph1, None, 10) + graphsToCheck = graphsToCheck :+ GraphToCheck(graph1, 1) + checkPersistence(graphsToCheck, 1) + + var iteration = 2 + while (iteration < 9) { + val graph = createGraph(sc) + checkpointer.updateGraph(graph) + graphsToCheck = graphsToCheck :+ GraphToCheck(graph, iteration) + checkPersistence(graphsToCheck, iteration) + iteration += 1 + } + } + + test("Checkpointing") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + val checkpointInterval = 2 + var graphsToCheck = Seq.empty[GraphToCheck] + + val graph1 = createGraph(sc) + val checkpointer = new PeriodicGraphCheckpointer(graph1, Some(path), checkpointInterval) + graph1.edges.count() + graph1.vertices.count() + graphsToCheck = graphsToCheck :+ GraphToCheck(graph1, 1) + checkCheckpoint(graphsToCheck, 1, checkpointInterval) + + var iteration = 2 + while (iteration < 9) { + val graph = createGraph(sc) + checkpointer.updateGraph(graph) + graph.vertices.count() + graph.edges.count() + graphsToCheck = graphsToCheck :+ GraphToCheck(graph, iteration) + checkCheckpoint(graphsToCheck, iteration, checkpointInterval) + iteration += 1 + } + + checkpointer.deleteAllCheckpoints() + graphsToCheck.foreach { graph => + confirmCheckpointRemoved(graph.graph) + } + + Utils.deleteRecursively(tempDir) + } +} + +private object PeriodicGraphCheckpointerSuite { + + case class GraphToCheck(graph: Graph[Double, Double], gIndex: Int) + + val edges = Seq( + Edge[Double](0, 1, 0), + Edge[Double](1, 2, 0), + Edge[Double](2, 3, 0), + Edge[Double](3, 4, 0)) + + def createGraph(sc: SparkContext): Graph[Double, Double] = { + Graph.fromEdges[Double, Double](sc.parallelize(edges), 0) + } + + def checkPersistence(graphs: Seq[GraphToCheck], iteration: Int): Unit = { + graphs.foreach { g => + checkPersistence(g.graph, g.gIndex, iteration) + } + } + + /** + * Check storage level of graph. + * @param gIndex Index of graph in order inserted into checkpointer (from 1). + * @param iteration Total number of graphs inserted into checkpointer. + */ + def checkPersistence(graph: Graph[_, _], gIndex: Int, iteration: Int): Unit = { + try { + if (gIndex + 2 < iteration) { + assert(graph.vertices.getStorageLevel == StorageLevel.NONE) + assert(graph.edges.getStorageLevel == StorageLevel.NONE) + } else { + assert(graph.vertices.getStorageLevel != StorageLevel.NONE) + assert(graph.edges.getStorageLevel != StorageLevel.NONE) + } + } catch { + case _: AssertionError => + throw new Exception(s"PeriodicGraphCheckpointerSuite.checkPersistence failed with:\n" + + s"\t gIndex = $gIndex\n" + + s"\t iteration = $iteration\n" + + s"\t graph.vertices.getStorageLevel = ${graph.vertices.getStorageLevel}\n" + + s"\t graph.edges.getStorageLevel = ${graph.edges.getStorageLevel}\n") + } + } + + def checkCheckpoint(graphs: Seq[GraphToCheck], iteration: Int, checkpointInterval: Int): Unit = { + graphs.reverse.foreach { g => + checkCheckpoint(g.graph, g.gIndex, iteration, checkpointInterval) + } + } + + def confirmCheckpointRemoved(graph: Graph[_, _]): Unit = { + // Note: We cannot check graph.isCheckpointed since that value is never updated. + // Instead, we check for the presence of the checkpoint files. + // This test should continue to work even after this graph.isCheckpointed issue + // is fixed (though it can then be simplified and not look for the files). + val fs = FileSystem.get(graph.vertices.sparkContext.hadoopConfiguration) + graph.getCheckpointFiles.foreach { checkpointFile => + assert(!fs.exists(new Path(checkpointFile)), + "Graph checkpoint file should have been removed") + } + } + + /** + * Check checkpointed status of graph. + * @param gIndex Index of graph in order inserted into checkpointer (from 1). + * @param iteration Total number of graphs inserted into checkpointer. + */ + def checkCheckpoint( + graph: Graph[_, _], + gIndex: Int, + iteration: Int, + checkpointInterval: Int): Unit = { + try { + if (gIndex % checkpointInterval == 0) { + // We allow 2 checkpoint intervals since we perform an action (checkpointing a second graph) + // only AFTER PeriodicGraphCheckpointer decides whether to remove the previous checkpoint. + if (iteration - 2 * checkpointInterval < gIndex && gIndex <= iteration) { + assert(graph.isCheckpointed, "Graph should be checkpointed") + assert(graph.getCheckpointFiles.length == 2, "Graph should have 2 checkpoint files") + } else { + confirmCheckpointRemoved(graph) + } + } else { + // Graph should never be checkpointed + assert(!graph.isCheckpointed, "Graph should never have been checkpointed") + assert(graph.getCheckpointFiles.length == 0, "Graph should not have any checkpoint files") + } + } catch { + case e: AssertionError => + throw new Exception(s"PeriodicGraphCheckpointerSuite.checkCheckpoint failed with:\n" + + s"\t gIndex = $gIndex\n" + + s"\t iteration = $iteration\n" + + s"\t checkpointInterval = $checkpointInterval\n" + + s"\t graph.isCheckpointed = ${graph.isCheckpointed}\n" + + s"\t graph.getCheckpointFiles = ${graph.getCheckpointFiles.mkString(", ")}\n" + + s" AssertionError message: ${e.getMessage}") + } + } + +} From 659329f9ee51ca8ae6232e07c45b5d9144d49667 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 3 Feb 2015 00:14:43 -0800 Subject: [PATCH 039/578] [minor] update streaming linear algorithms Author: Xiangrui Meng Closes #4329 from mengxr/streaming-lr and squashes the following commits: 78731e1 [Xiangrui Meng] update streaming linear algorithms --- .../StreamingLogisticRegressionWithSGD.scala | 3 +- .../regression/StreamingLinearAlgorithm.scala | 41 ++++++++++--------- .../StreamingLinearRegressionWithSGD.scala | 2 +- 3 files changed, 24 insertions(+), 22 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 eabd2162e287f..6a3893d0e41d2 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 @@ -88,8 +88,7 @@ class StreamingLogisticRegressionWithSGD private[mllib] ( /** Set the initial weights. Default: [0.0, 0.0]. */ def setInitialWeights(initialWeights: Vector): this.type = { - this.model = Option(algorithm.createModel(initialWeights, 0.0)) + this.model = Some(algorithm.createModel(initialWeights, 0.0)) this } - } 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 39a0dee931d3d..44a8dbb994cfb 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 @@ -21,7 +21,7 @@ import scala.reflect.ClassTag import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.streaming.dstream.DStream /** @@ -58,7 +58,7 @@ abstract class StreamingLinearAlgorithm[ A <: GeneralizedLinearAlgorithm[M]] extends Logging { /** The model to be updated and used for prediction. */ - protected var model: Option[M] = null + protected var model: Option[M] = None /** The algorithm to use for updating. */ protected val algorithm: A @@ -77,18 +77,25 @@ abstract class StreamingLinearAlgorithm[ * @param data DStream containing labeled data */ def trainOn(data: DStream[LabeledPoint]) { - if (Option(model) == None) { - logError("Model must be initialized before starting training") - throw new IllegalArgumentException + if (model.isEmpty) { + throw new IllegalArgumentException("Model must be initialized before starting training.") } data.foreachRDD { (rdd, time) => - model = Option(algorithm.run(rdd, model.get.weights)) - logInfo("Model updated at time %s".format(time.toString)) - val display = model.get.weights.size match { - case x if x > 100 => model.get.weights.toArray.take(100).mkString("[", ",", "...") - case _ => model.get.weights.toArray.mkString("[", ",", "]") + val initialWeights = + model match { + case Some(m) => + m.weights + case None => + val numFeatures = rdd.first().features.size + Vectors.dense(numFeatures) } - logInfo("Current model: weights, %s".format (display)) + model = Some(algorithm.run(rdd, initialWeights)) + logInfo("Model updated at time %s".format(time.toString)) + val display = model.get.weights.size match { + case x if x > 100 => model.get.weights.toArray.take(100).mkString("[", ",", "...") + case _ => model.get.weights.toArray.mkString("[", ",", "]") + } + logInfo("Current model: weights, %s".format (display)) } } @@ -99,10 +106,8 @@ abstract class StreamingLinearAlgorithm[ * @return DStream containing predictions */ def predictOn(data: DStream[Vector]): DStream[Double] = { - if (Option(model) == None) { - val msg = "Model must be initialized before starting prediction" - logError(msg) - throw new IllegalArgumentException(msg) + if (model.isEmpty) { + throw new IllegalArgumentException("Model must be initialized before starting prediction.") } data.map(model.get.predict) } @@ -114,10 +119,8 @@ abstract class StreamingLinearAlgorithm[ * @return DStream containing the input keys and the predictions as values */ def predictOnValues[K: ClassTag](data: DStream[(K, Vector)]): DStream[(K, Double)] = { - if (Option(model) == None) { - val msg = "Model must be initialized before starting prediction" - logError(msg) - throw new IllegalArgumentException(msg) + if (model.isEmpty) { + throw new IllegalArgumentException("Model must be initialized before starting prediction") } data.mapValues(model.get.predict) } 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 c0625b4880953..e5e6301127a28 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 @@ -79,7 +79,7 @@ class StreamingLinearRegressionWithSGD private[mllib] ( /** Set the initial weights. Default: [0.0, 0.0]. */ def setInitialWeights(initialWeights: Vector): this.type = { - this.model = Option(algorithm.createModel(initialWeights, 0.0)) + this.model = Some(algorithm.createModel(initialWeights, 0.0)) this } From 37df330135e6a3e62c580e5706eade5f1eaf5b13 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 3 Feb 2015 00:29:04 -0800 Subject: [PATCH 040/578] [SQL][DataFrame] Remove DataFrameApi, ExpressionApi, and GroupedDataFrameApi They were there mostly for code review and easier check of the API. I don't think they need to be there anymore. Author: Reynold Xin Closes #4328 from rxin/remove-df-api and squashes the following commits: 723d600 [Reynold Xin] [SQL][DataFrame] Remove DataFrameApi and ColumnApi. --- .../scala/org/apache/spark/sql/Column.scala | 118 +++---- .../org/apache/spark/sql/DataFrame.scala | 110 +++--- .../org/apache/spark/sql/DataFrameImpl.scala | 8 +- .../apache/spark/sql/GroupedDataFrame.scala | 19 +- .../scala/org/apache/spark/sql/RDDApi.scala | 63 ++++ .../main/scala/org/apache/spark/sql/api.scala | 326 ------------------ 6 files changed, 200 insertions(+), 444 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/api.scala 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 6f48d7c3fe1b6..0d6055ff232b1 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 @@ -56,7 +56,7 @@ private[sql] object Column { * */ // TODO: Improve documentation. -trait Column extends DataFrame with ExpressionApi { +trait Column extends DataFrame { protected[sql] def expr: Expression @@ -101,7 +101,7 @@ trait Column extends DataFrame with ExpressionApi { * df.select( -df("amount") ) * }}} */ - override def unary_- : Column = constructColumn(null) { UnaryMinus(expr) } + def unary_- : Column = constructColumn(null) { UnaryMinus(expr) } /** * Bitwise NOT. @@ -110,7 +110,7 @@ trait Column extends DataFrame with ExpressionApi { * df.select( ~df("flags") ) * }}} */ - override def unary_~ : Column = constructColumn(null) { BitwiseNot(expr) } + def unary_~ : Column = constructColumn(null) { BitwiseNot(expr) } /** * Inversion of boolean expression, i.e. NOT. @@ -119,7 +119,7 @@ trait Column extends DataFrame with ExpressionApi { * df.select( !df("isActive") ) * }} */ - override def unary_! : Column = constructColumn(null) { Not(expr) } + def unary_! : Column = constructColumn(null) { Not(expr) } /** @@ -130,7 +130,7 @@ trait Column extends DataFrame with ExpressionApi { * df.select( df("colA".equalTo(df("colB")) ) * }}} */ - override def === (other: Column): Column = constructColumn(other) { + def === (other: Column): Column = constructColumn(other) { EqualTo(expr, other.expr) } @@ -142,7 +142,7 @@ trait Column extends DataFrame with ExpressionApi { * df.select( df("colA".equalTo("Zaharia") ) * }}} */ - override def === (literal: Any): Column = this === lit(literal) + def === (literal: Any): Column = this === lit(literal) /** * Equality test with an expression. @@ -152,7 +152,7 @@ trait Column extends DataFrame with ExpressionApi { * df.select( df("colA".equalTo(df("colB")) ) * }}} */ - override def equalTo(other: Column): Column = this === other + def equalTo(other: Column): Column = this === other /** * Equality test with a literal value. @@ -162,7 +162,7 @@ trait Column extends DataFrame with ExpressionApi { * df.select( df("colA".equalTo("Zaharia") ) * }}} */ - override def equalTo(literal: Any): Column = this === literal + def equalTo(literal: Any): Column = this === literal /** * Inequality test with an expression. @@ -172,7 +172,7 @@ trait Column extends DataFrame with ExpressionApi { * df.select( !(df("colA") === df("colB")) ) * }}} */ - override def !== (other: Column): Column = constructColumn(other) { + def !== (other: Column): Column = constructColumn(other) { Not(EqualTo(expr, other.expr)) } @@ -184,7 +184,7 @@ trait Column extends DataFrame with ExpressionApi { * df.select( !(df("colA") === 15) ) * }}} */ - override def !== (literal: Any): Column = this !== lit(literal) + def !== (literal: Any): Column = this !== lit(literal) /** * Greater than an expression. @@ -193,7 +193,7 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("age") > Literal(21) ) * }}} */ - override def > (other: Column): Column = constructColumn(other) { + def > (other: Column): Column = constructColumn(other) { GreaterThan(expr, other.expr) } @@ -204,7 +204,7 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("age") > 21 ) * }}} */ - override def > (literal: Any): Column = this > lit(literal) + def > (literal: Any): Column = this > lit(literal) /** * Less than an expression. @@ -213,7 +213,7 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("age") < Literal(21) ) * }}} */ - override def < (other: Column): Column = constructColumn(other) { + def < (other: Column): Column = constructColumn(other) { LessThan(expr, other.expr) } @@ -224,7 +224,7 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("age") < 21 ) * }}} */ - override def < (literal: Any): Column = this < lit(literal) + def < (literal: Any): Column = this < lit(literal) /** * Less than or equal to an expression. @@ -233,7 +233,7 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("age") <= Literal(21) ) * }}} */ - override def <= (other: Column): Column = constructColumn(other) { + def <= (other: Column): Column = constructColumn(other) { LessThanOrEqual(expr, other.expr) } @@ -244,7 +244,7 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("age") <= 21 ) * }}} */ - override def <= (literal: Any): Column = this <= lit(literal) + def <= (literal: Any): Column = this <= lit(literal) /** * Greater than or equal to an expression. @@ -253,7 +253,7 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("age") >= Literal(21) ) * }}} */ - override def >= (other: Column): Column = constructColumn(other) { + def >= (other: Column): Column = constructColumn(other) { GreaterThanOrEqual(expr, other.expr) } @@ -264,12 +264,12 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("age") >= 21 ) * }}} */ - override def >= (literal: Any): Column = this >= lit(literal) + def >= (literal: Any): Column = this >= lit(literal) /** * Equality test with an expression that is safe for null values. */ - override def <=> (other: Column): Column = constructColumn(other) { + def <=> (other: Column): Column = constructColumn(other) { other match { case null => EqualNullSafe(expr, lit(null).expr) case _ => EqualNullSafe(expr, other.expr) @@ -279,17 +279,17 @@ trait Column extends DataFrame with ExpressionApi { /** * Equality test with a literal value that is safe for null values. */ - override def <=> (literal: Any): Column = this <=> lit(literal) + def <=> (literal: Any): Column = this <=> lit(literal) /** * True if the current expression is null. */ - override def isNull: Column = constructColumn(null) { IsNull(expr) } + def isNull: Column = constructColumn(null) { IsNull(expr) } /** * True if the current expression is NOT null. */ - override def isNotNull: Column = constructColumn(null) { IsNotNull(expr) } + def isNotNull: Column = constructColumn(null) { IsNotNull(expr) } /** * Boolean OR with an expression. @@ -298,7 +298,7 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("inSchool") || people("isEmployed") ) * }}} */ - override def || (other: Column): Column = constructColumn(other) { + def || (other: Column): Column = constructColumn(other) { Or(expr, other.expr) } @@ -309,7 +309,7 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("inSchool") || true ) * }}} */ - override def || (literal: Boolean): Column = this || lit(literal) + def || (literal: Boolean): Column = this || lit(literal) /** * Boolean AND with an expression. @@ -318,7 +318,7 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("inSchool") && people("isEmployed") ) * }}} */ - override def && (other: Column): Column = constructColumn(other) { + def && (other: Column): Column = constructColumn(other) { And(expr, other.expr) } @@ -329,43 +329,43 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("inSchool") && true ) * }}} */ - override def && (literal: Boolean): Column = this && lit(literal) + def && (literal: Boolean): Column = this && lit(literal) /** * Bitwise AND with an expression. */ - override def & (other: Column): Column = constructColumn(other) { + def & (other: Column): Column = constructColumn(other) { BitwiseAnd(expr, other.expr) } /** * Bitwise AND with a literal value. */ - override def & (literal: Any): Column = this & lit(literal) + def & (literal: Any): Column = this & lit(literal) /** * Bitwise OR with an expression. */ - override def | (other: Column): Column = constructColumn(other) { + def | (other: Column): Column = constructColumn(other) { BitwiseOr(expr, other.expr) } /** * Bitwise OR with a literal value. */ - override def | (literal: Any): Column = this | lit(literal) + def | (literal: Any): Column = this | lit(literal) /** * Bitwise XOR with an expression. */ - override def ^ (other: Column): Column = constructColumn(other) { + def ^ (other: Column): Column = constructColumn(other) { BitwiseXor(expr, other.expr) } /** * Bitwise XOR with a literal value. */ - override def ^ (literal: Any): Column = this ^ lit(literal) + def ^ (literal: Any): Column = this ^ lit(literal) /** * Sum of this expression and another expression. @@ -374,7 +374,7 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("height") + people("weight") ) * }}} */ - override def + (other: Column): Column = constructColumn(other) { + def + (other: Column): Column = constructColumn(other) { Add(expr, other.expr) } @@ -385,7 +385,7 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("height") + 10 ) * }}} */ - override def + (literal: Any): Column = this + lit(literal) + def + (literal: Any): Column = this + lit(literal) /** * Subtraction. Subtract the other expression from this expression. @@ -394,7 +394,7 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("height") - people("weight") ) * }}} */ - override def - (other: Column): Column = constructColumn(other) { + def - (other: Column): Column = constructColumn(other) { Subtract(expr, other.expr) } @@ -405,7 +405,7 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("height") - 10 ) * }}} */ - override def - (literal: Any): Column = this - lit(literal) + def - (literal: Any): Column = this - lit(literal) /** * Multiplication of this expression and another expression. @@ -414,7 +414,7 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("height") * people("weight") ) * }}} */ - override def * (other: Column): Column = constructColumn(other) { + def * (other: Column): Column = constructColumn(other) { Multiply(expr, other.expr) } @@ -425,7 +425,7 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("height") * 10 ) * }}} */ - override def * (literal: Any): Column = this * lit(literal) + def * (literal: Any): Column = this * lit(literal) /** * Division this expression by another expression. @@ -434,7 +434,7 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("height") / people("weight") ) * }}} */ - override def / (other: Column): Column = constructColumn(other) { + def / (other: Column): Column = constructColumn(other) { Divide(expr, other.expr) } @@ -445,19 +445,19 @@ trait Column extends DataFrame with ExpressionApi { * people.select( people("height") / 10 ) * }}} */ - override def / (literal: Any): Column = this / lit(literal) + def / (literal: Any): Column = this / lit(literal) /** * Modulo (a.k.a. remainder) expression. */ - override def % (other: Column): Column = constructColumn(other) { + def % (other: Column): Column = constructColumn(other) { Remainder(expr, other.expr) } /** * Modulo (a.k.a. remainder) expression. */ - override def % (literal: Any): Column = this % lit(literal) + def % (literal: Any): Column = this % lit(literal) /** @@ -465,29 +465,29 @@ trait Column extends DataFrame with ExpressionApi { * by the evaluated values of the arguments. */ @scala.annotation.varargs - override def in(list: Column*): Column = { + def in(list: Column*): Column = { new IncomputableColumn(In(expr, list.map(_.expr))) } - override def like(literal: String): Column = constructColumn(null) { + def like(literal: String): Column = constructColumn(null) { Like(expr, lit(literal).expr) } - override def rlike(literal: String): Column = constructColumn(null) { + def rlike(literal: String): Column = constructColumn(null) { RLike(expr, lit(literal).expr) } /** * An expression that gets an item at position `ordinal` out of an array. */ - override def getItem(ordinal: Int): Column = constructColumn(null) { + def getItem(ordinal: Int): Column = constructColumn(null) { GetItem(expr, Literal(ordinal)) } /** * An expression that gets a field by name in a [[StructField]]. */ - override def getField(fieldName: String): Column = constructColumn(null) { + def getField(fieldName: String): Column = constructColumn(null) { GetField(expr, fieldName) } @@ -496,7 +496,7 @@ trait Column extends DataFrame with ExpressionApi { * @param startPos expression for the starting position. * @param len expression for the length of the substring. */ - override def substr(startPos: Column, len: Column): Column = { + def substr(startPos: Column, len: Column): Column = { new IncomputableColumn(Substring(expr, startPos.expr, len.expr)) } @@ -505,25 +505,25 @@ trait Column extends DataFrame with ExpressionApi { * @param startPos starting position. * @param len length of the substring. */ - override def substr(startPos: Int, len: Int): Column = this.substr(lit(startPos), lit(len)) + def substr(startPos: Int, len: Int): Column = this.substr(lit(startPos), lit(len)) - override def contains(other: Column): Column = constructColumn(other) { + def contains(other: Column): Column = constructColumn(other) { Contains(expr, other.expr) } - override def contains(literal: Any): Column = this.contains(lit(literal)) + def contains(literal: Any): Column = this.contains(lit(literal)) - override def startsWith(other: Column): Column = constructColumn(other) { + def startsWith(other: Column): Column = constructColumn(other) { StartsWith(expr, other.expr) } - override def startsWith(literal: String): Column = this.startsWith(lit(literal)) + def startsWith(literal: String): Column = this.startsWith(lit(literal)) - override def endsWith(other: Column): Column = constructColumn(other) { + def endsWith(other: Column): Column = constructColumn(other) { EndsWith(expr, other.expr) } - override def endsWith(literal: String): Column = this.endsWith(lit(literal)) + def endsWith(literal: String): Column = this.endsWith(lit(literal)) /** * Gives the column an alias. @@ -545,7 +545,7 @@ trait Column extends DataFrame with ExpressionApi { * df.select(df("colA").cast("int")) * }}} */ - override def cast(to: DataType): Column = constructColumn(null) { Cast(expr, to) } + def cast(to: DataType): Column = constructColumn(null) { Cast(expr, to) } /** * Casts the column to a different data type, using the canonical string representation @@ -556,7 +556,7 @@ trait Column extends DataFrame with ExpressionApi { * df.select(df("colA").cast("int")) * }}} */ - override def cast(to: String): Column = constructColumn(null) { + def cast(to: String): Column = constructColumn(null) { Cast(expr, to.toLowerCase match { case "string" => StringType case "boolean" => BooleanType @@ -573,9 +573,9 @@ trait Column extends DataFrame with ExpressionApi { }) } - override def desc: Column = constructColumn(null) { SortOrder(expr, Descending) } + def desc: Column = constructColumn(null) { SortOrder(expr, Descending) } - override def asc: Column = constructColumn(null) { SortOrder(expr, Ascending) } + def asc: Column = constructColumn(null) { SortOrder(expr, Ascending) } } 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 4cbfb6af5de94..5920852e8c178 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 @@ -71,7 +71,7 @@ private[sql] object DataFrame { * }}} */ // TODO: Improve documentation. -trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { +trait DataFrame extends RDDApi[Row] { val sqlContext: SQLContext @@ -80,7 +80,7 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { protected[sql] def logicalPlan: LogicalPlan - /** Left here for compatibility reasons. */ + /** Left here for backward compatibility. */ @deprecated("1.3.0", "use toDataFrame") def toSchemaRDD: DataFrame = this @@ -102,16 +102,16 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { def toDataFrame(colName: String, colNames: String*): DataFrame /** Returns the schema of this [[DataFrame]]. */ - override def schema: StructType + def schema: StructType /** Returns all column names and their data types as an array. */ - override def dtypes: Array[(String, String)] + def dtypes: Array[(String, String)] /** Returns all column names as an array. */ - override def columns: Array[String] = schema.fields.map(_.name) + def columns: Array[String] = schema.fields.map(_.name) /** Prints the schema to the console in a nice tree format. */ - override def printSchema(): Unit + def printSchema(): Unit /** * Cartesian join with another [[DataFrame]]. @@ -120,7 +120,7 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * * @param right Right side of the join operation. */ - override def join(right: DataFrame): DataFrame + def join(right: DataFrame): DataFrame /** * Inner join with another [[DataFrame]], using the given join expression. @@ -131,7 +131,7 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * df1.join(df2).where($"df1Key" === $"df2Key") * }}} */ - override def join(right: DataFrame, joinExprs: Column): DataFrame + def join(right: DataFrame, joinExprs: Column): DataFrame /** * Join with another [[DataFrame]], usin g the given join expression. The following performs @@ -145,7 +145,7 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * @param joinExprs Join expression. * @param joinType One of: `inner`, `outer`, `left_outer`, `right_outer`, `semijoin`. */ - override def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame + def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame /** * Returns a new [[DataFrame]] sorted by the specified column, all in ascending order. @@ -157,7 +157,7 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * }}} */ @scala.annotation.varargs - override def sort(sortCol: String, sortCols: String*): DataFrame + def sort(sortCol: String, sortCols: String*): DataFrame /** * Returns a new [[DataFrame]] sorted by the given expressions. For example: @@ -166,26 +166,26 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * }}} */ @scala.annotation.varargs - override def sort(sortExpr: Column, sortExprs: Column*): DataFrame + def sort(sortExpr: Column, sortExprs: Column*): DataFrame /** * Returns a new [[DataFrame]] sorted by the given expressions. * This is an alias of the `sort` function. */ @scala.annotation.varargs - override def orderBy(sortCol: String, sortCols: String*): DataFrame + def orderBy(sortCol: String, sortCols: String*): DataFrame /** * Returns a new [[DataFrame]] sorted by the given expressions. * This is an alias of the `sort` function. */ @scala.annotation.varargs - override def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame + def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame /** * Selects column based on the column name and return it as a [[Column]]. */ - override def apply(colName: String): Column + def apply(colName: String): Column /** * Selects a set of expressions, wrapped in a Product. @@ -195,12 +195,12 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * df.select($"colA", $"colB" + 1) * }}} */ - override def apply(projection: Product): DataFrame + def apply(projection: Product): DataFrame /** * Returns a new [[DataFrame]] with an alias set. */ - override def as(name: String): DataFrame + def as(name: String): DataFrame /** * Selects a set of expressions. @@ -209,7 +209,7 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * }}} */ @scala.annotation.varargs - override def select(cols: Column*): DataFrame + def select(cols: Column*): DataFrame /** * Selects a set of columns. This is a variant of `select` that can only select @@ -222,7 +222,7 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * }}} */ @scala.annotation.varargs - override def select(col: String, cols: String*): DataFrame + def select(col: String, cols: String*): DataFrame /** * Filters rows using the given condition. @@ -233,7 +233,7 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * peopleDf($"age" > 15) * }}} */ - override def filter(condition: Column): DataFrame + def filter(condition: Column): DataFrame /** * Filters rows using the given condition. This is an alias for `filter`. @@ -244,7 +244,7 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * peopleDf($"age" > 15) * }}} */ - override def where(condition: Column): DataFrame + def where(condition: Column): DataFrame /** * Filters rows using the given condition. This is a shorthand meant for Scala. @@ -255,7 +255,7 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * peopleDf($"age" > 15) * }}} */ - override def apply(condition: Column): DataFrame + def apply(condition: Column): DataFrame /** * Groups the [[DataFrame]] using the specified columns, so we can run aggregation on them. @@ -273,7 +273,7 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * }}} */ @scala.annotation.varargs - override def groupBy(cols: Column*): GroupedDataFrame + def groupBy(cols: Column*): GroupedDataFrame /** * Groups the [[DataFrame]] using the specified columns, so we can run aggregation on them. @@ -294,7 +294,7 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * }}} */ @scala.annotation.varargs - override def groupBy(col1: String, cols: String*): GroupedDataFrame + def groupBy(col1: String, cols: String*): GroupedDataFrame /** * Aggregates on the entire [[DataFrame]] without groups. @@ -304,7 +304,7 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * df.groupBy().agg(Map("age" -> "max", "salary" -> "avg")) * }} */ - override def agg(exprs: Map[String, String]): DataFrame + def agg(exprs: Map[String, String]): DataFrame /** * Aggregates on the entire [[DataFrame]] without groups. @@ -314,7 +314,7 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * df.groupBy().agg(Map("age" -> "max", "salary" -> "avg")) * }} */ - override def agg(exprs: java.util.Map[String, String]): DataFrame + def agg(exprs: java.util.Map[String, String]): DataFrame /** * Aggregates on the entire [[DataFrame]] without groups. @@ -325,31 +325,31 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * }} */ @scala.annotation.varargs - override def agg(expr: Column, exprs: Column*): DataFrame + def agg(expr: Column, exprs: Column*): DataFrame /** * 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]]. */ - override def limit(n: Int): DataFrame + 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. */ - override def unionAll(other: DataFrame): DataFrame + 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. */ - override def intersect(other: DataFrame): DataFrame + 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. */ - override def except(other: DataFrame): DataFrame + def except(other: DataFrame): DataFrame /** * Returns a new [[DataFrame]] by sampling a fraction of rows. @@ -358,7 +358,7 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * @param fraction Fraction of rows to generate. * @param seed Seed for sampling. */ - override def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame + def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame /** * Returns a new [[DataFrame]] by sampling a fraction of rows, using a random seed. @@ -366,24 +366,24 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * @param withReplacement Sample with replacement or not. * @param fraction Fraction of rows to generate. */ - override def sample(withReplacement: Boolean, fraction: Double): DataFrame + def sample(withReplacement: Boolean, fraction: Double): DataFrame ///////////////////////////////////////////////////////////////////////////// /** * Returns a new [[DataFrame]] by adding a column. */ - override def addColumn(colName: String, col: Column): DataFrame + def addColumn(colName: String, col: Column): DataFrame /** * Returns the first `n` rows. */ - override def head(n: Int): Array[Row] + def head(n: Int): Array[Row] /** * Returns the first row. */ - override def head(): Row + def head(): Row /** * Returns the first row. Alias for head(). @@ -453,7 +453,17 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { /** * Returns the content of the [[DataFrame]] as an [[RDD]] of [[Row]]s. */ - override def rdd: RDD[Row] + def rdd: RDD[Row] + + /** + * Returns the content of the [[DataFrame]] as a [[JavaRDD]] of [[Row]]s. + */ + def toJavaRDD: JavaRDD[Row] = rdd.toJavaRDD() + + /** + * Returns the content of the [[DataFrame]] as a [[JavaRDD]] of [[Row]]s. + */ + def javaRDD: JavaRDD[Row] = toJavaRDD /** * Registers this RDD as a temporary table using the given name. The lifetime of this temporary @@ -461,14 +471,14 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * * @group schema */ - override def registerTempTable(tableName: String): Unit + def registerTempTable(tableName: String): Unit /** * 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]]. */ - override def saveAsParquetFile(path: String): Unit + def saveAsParquetFile(path: String): Unit /** * :: Experimental :: @@ -481,7 +491,7 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * be the target of an `insertInto`. */ @Experimental - override def saveAsTable(tableName: String): Unit + def saveAsTable(tableName: String): Unit /** * :: Experimental :: @@ -494,7 +504,7 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * be the target of an `insertInto`. */ @Experimental - override def saveAsTable( + def saveAsTable( tableName: String, dataSourceName: String, option: (String, String), @@ -511,22 +521,22 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * be the target of an `insertInto`. */ @Experimental - override def saveAsTable( + def saveAsTable( tableName: String, dataSourceName: String, options: java.util.Map[String, String]): Unit @Experimental - override def save(path: String): Unit + def save(path: String): Unit @Experimental - override def save( + def save( dataSourceName: String, option: (String, String), options: (String, String)*): Unit @Experimental - override def save( + def save( dataSourceName: String, options: java.util.Map[String, String]): Unit @@ -535,12 +545,20 @@ trait DataFrame extends DataFrameSpecificApi with RDDApi[Row] { * Adds the rows from this RDD to the specified table, optionally overwriting the existing data. */ @Experimental - override def insertInto(tableName: String, overwrite: Boolean): Unit + def insertInto(tableName: String, overwrite: Boolean): Unit + + /** + * :: Experimental :: + * Adds the rows from this RDD to the specified table. + * Throws an exception if the table already exists. + */ + @Experimental + def insertInto(tableName: String): Unit = insertInto(tableName, overwrite = false) /** * Returns the content of the [[DataFrame]] as a RDD of JSON strings. */ - override def toJSON: RDD[String] + def toJSON: RDD[String] //////////////////////////////////////////////////////////////////////////// // 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 f84dbf32fa5b2..49fd131534bb5 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,8 +17,6 @@ package org.apache.spark.sql -import java.util.{List => JList} - import scala.language.implicitConversions import scala.reflect.ClassTag import scala.collection.JavaConversions._ @@ -42,13 +40,17 @@ import org.apache.spark.util.Utils /** - * See [[DataFrame]] for documentation. + * 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) extends DataFrame { + /** + * A constructor that automatically analyzes the logical plan. This reports error eagerly + * as the [[DataFrame]] is constructed. + */ def this(sqlContext: SQLContext, logicalPlan: LogicalPlan) = { this(sqlContext, { val qe = sqlContext.executePlan(logicalPlan) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala index d3acd41bbf3eb..6d0f3e8ce37cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala @@ -30,8 +30,7 @@ import org.apache.spark.sql.catalyst.plans.logical.Aggregate /** * A set of methods for aggregations on a [[DataFrame]], created by [[DataFrame.groupBy]]. */ -class GroupedDataFrame protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expression]) - extends GroupedDataFrameApi { +class GroupedDataFrame protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expression]) { private[this] implicit def toDataFrame(aggExprs: Seq[NamedExpression]): DataFrame = { val namedGroupingExprs = groupingExprs.map { @@ -72,7 +71,7 @@ class GroupedDataFrame protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expr * )) * }}} */ - override def agg(exprs: Map[String, String]): DataFrame = { + def agg(exprs: Map[String, String]): DataFrame = { exprs.map { case (colName, expr) => val a = strToExpr(expr)(df(colName).expr) Alias(a, a.toString)() @@ -109,7 +108,7 @@ class GroupedDataFrame protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expr * }}} */ @scala.annotation.varargs - override def agg(expr: Column, exprs: Column*): DataFrame = { + 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)() @@ -121,35 +120,35 @@ class GroupedDataFrame protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expr * Count the number of rows for each group. * The resulting [[DataFrame]] will also contain the grouping columns. */ - override def count(): DataFrame = Seq(Alias(Count(LiteralExpr(1)), "count")()) + def count(): DataFrame = Seq(Alias(Count(LiteralExpr(1)), "count")()) /** * 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. */ - override def mean(): DataFrame = aggregateNumericColumns(Average) + def mean(): DataFrame = aggregateNumericColumns(Average) /** * Compute the max value for each numeric columns for each group. * The resulting [[DataFrame]] will also contain the grouping columns. */ - override def max(): DataFrame = aggregateNumericColumns(Max) + def max(): DataFrame = aggregateNumericColumns(Max) /** * Compute the mean value for each numeric columns for each group. * The resulting [[DataFrame]] will also contain the grouping columns. */ - override def avg(): DataFrame = aggregateNumericColumns(Average) + def avg(): DataFrame = aggregateNumericColumns(Average) /** * Compute the min value for each numeric column for each group. * The resulting [[DataFrame]] will also contain the grouping columns. */ - override def min(): DataFrame = aggregateNumericColumns(Min) + def min(): DataFrame = aggregateNumericColumns(Min) /** * Compute the sum for each numeric columns for each group. * The resulting [[DataFrame]] will also contain the grouping columns. */ - override def sum(): DataFrame = aggregateNumericColumns(Sum) + def sum(): DataFrame = aggregateNumericColumns(Sum) } 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 new file mode 100644 index 0000000000000..38e6382f171d5 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.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.sql + +import scala.reflect.ClassTag + +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel + + +/** + * An internal interface defining the RDD-like methods for [[DataFrame]]. + * Please use [[DataFrame]] directly, and do NOT use this. + */ +private[sql] trait RDDApi[T] { + + def cache(): this.type = persist() + + def persist(): this.type + + def persist(newLevel: StorageLevel): this.type + + def unpersist(): this.type = unpersist(blocking = false) + + def unpersist(blocking: Boolean): this.type + + def map[R: ClassTag](f: T => R): RDD[R] + + def flatMap[R: ClassTag](f: T => TraversableOnce[R]): RDD[R] + + def mapPartitions[R: ClassTag](f: Iterator[T] => Iterator[R]): RDD[R] + + def foreach(f: T => Unit): Unit + + def foreachPartition(f: Iterator[T] => Unit): Unit + + def take(n: Int): Array[T] + + def collect(): Array[T] + + def collectAsList(): java.util.List[T] + + def count(): Long + + def first(): T + + def repartition(numPartitions: Int): DataFrame +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api.scala b/sql/core/src/main/scala/org/apache/spark/sql/api.scala deleted file mode 100644 index c4a00cdb20408..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/api.scala +++ /dev/null @@ -1,326 +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 org.apache.spark.annotation.Experimental -import org.apache.spark.api.java.JavaRDD -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.types.{DataType, StructType} -import org.apache.spark.storage.StorageLevel - - -/** - * An internal interface defining the RDD-like methods for [[DataFrame]]. - * Please use [[DataFrame]] directly, and do NOT use this. - */ -private[sql] trait RDDApi[T] { - - def cache(): this.type = persist() - - def persist(): this.type - - def persist(newLevel: StorageLevel): this.type - - def unpersist(): this.type = unpersist(blocking = false) - - def unpersist(blocking: Boolean): this.type - - def map[R: ClassTag](f: T => R): RDD[R] - - def flatMap[R: ClassTag](f: T => TraversableOnce[R]): RDD[R] - - def mapPartitions[R: ClassTag](f: Iterator[T] => Iterator[R]): RDD[R] - - def foreach(f: T => Unit): Unit - - def foreachPartition(f: Iterator[T] => Unit): Unit - - def take(n: Int): Array[T] - - def collect(): Array[T] - - def collectAsList(): java.util.List[T] - - def count(): Long - - def first(): T - - def repartition(numPartitions: Int): DataFrame -} - - -/** - * An internal interface defining data frame related methods in [[DataFrame]]. - * Please use [[DataFrame]] directly, and do NOT use this. - */ -private[sql] trait DataFrameSpecificApi { - - def schema: StructType - - def printSchema(): Unit - - def dtypes: Array[(String, String)] - - def columns: Array[String] - - def head(): Row - - def head(n: Int): Array[Row] - - ///////////////////////////////////////////////////////////////////////////// - // Relational operators - ///////////////////////////////////////////////////////////////////////////// - def apply(colName: String): Column - - def apply(projection: Product): DataFrame - - @scala.annotation.varargs - def select(cols: Column*): DataFrame - - @scala.annotation.varargs - def select(col: String, cols: String*): DataFrame - - def apply(condition: Column): DataFrame - - def as(name: String): DataFrame - - def filter(condition: Column): DataFrame - - def where(condition: Column): DataFrame - - @scala.annotation.varargs - def groupBy(cols: Column*): GroupedDataFrame - - @scala.annotation.varargs - def groupBy(col1: String, cols: String*): GroupedDataFrame - - def agg(exprs: Map[String, String]): DataFrame - - def agg(exprs: java.util.Map[String, String]): DataFrame - - @scala.annotation.varargs - def agg(expr: Column, exprs: Column*): DataFrame - - @scala.annotation.varargs - def sort(sortExpr: Column, sortExprs: Column*): DataFrame - - @scala.annotation.varargs - def sort(sortCol: String, sortCols: String*): DataFrame - - @scala.annotation.varargs - def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame - - @scala.annotation.varargs - def orderBy(sortCol: String, sortCols: String*): DataFrame - - def join(right: DataFrame): DataFrame - - def join(right: DataFrame, joinExprs: Column): DataFrame - - def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame - - def limit(n: Int): DataFrame - - def unionAll(other: DataFrame): DataFrame - - def intersect(other: DataFrame): DataFrame - - def except(other: DataFrame): DataFrame - - def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame - - def sample(withReplacement: Boolean, fraction: Double): DataFrame - - ///////////////////////////////////////////////////////////////////////////// - // Column mutation - ///////////////////////////////////////////////////////////////////////////// - def addColumn(colName: String, col: Column): DataFrame - - ///////////////////////////////////////////////////////////////////////////// - // I/O and interaction with other frameworks - ///////////////////////////////////////////////////////////////////////////// - - def rdd: RDD[Row] - - def toJavaRDD: JavaRDD[Row] = rdd.toJavaRDD() - - def toJSON: RDD[String] - - def registerTempTable(tableName: String): Unit - - def saveAsParquetFile(path: String): Unit - - @Experimental - def saveAsTable(tableName: String): Unit - - @Experimental - def saveAsTable( - tableName: String, - dataSourceName: String, - option: (String, String), - options: (String, String)*): Unit - - @Experimental - def saveAsTable( - tableName: String, - dataSourceName: String, - options: java.util.Map[String, String]): Unit - - @Experimental - def save(path: String): Unit - - @Experimental - def save( - dataSourceName: String, - option: (String, String), - options: (String, String)*): Unit - - @Experimental - def save( - dataSourceName: String, - options: java.util.Map[String, String]): Unit - - @Experimental - def insertInto(tableName: String, overwrite: Boolean): Unit - - @Experimental - def insertInto(tableName: String): Unit = insertInto(tableName, overwrite = false) - - ///////////////////////////////////////////////////////////////////////////// - // Stat functions - ///////////////////////////////////////////////////////////////////////////// -// def describe(): Unit -// -// def mean(): Unit -// -// def max(): Unit -// -// def min(): Unit -} - - -/** - * An internal interface defining expression APIs for [[DataFrame]]. - * Please use [[DataFrame]] and [[Column]] directly, and do NOT use this. - */ -private[sql] trait ExpressionApi { - - def isComputable: Boolean - - def unary_- : Column - def unary_! : Column - def unary_~ : Column - - def + (other: Column): Column - def + (other: Any): Column - def - (other: Column): Column - def - (other: Any): Column - def * (other: Column): Column - def * (other: Any): Column - def / (other: Column): Column - def / (other: Any): Column - def % (other: Column): Column - def % (other: Any): Column - def & (other: Column): Column - def & (other: Any): Column - def | (other: Column): Column - def | (other: Any): Column - def ^ (other: Column): Column - def ^ (other: Any): Column - - def && (other: Column): Column - def && (other: Boolean): Column - def || (other: Column): Column - def || (other: Boolean): Column - - def < (other: Column): Column - def < (other: Any): Column - def <= (other: Column): Column - def <= (other: Any): Column - def > (other: Column): Column - def > (other: Any): Column - def >= (other: Column): Column - def >= (other: Any): Column - def === (other: Column): Column - def === (other: Any): Column - def equalTo(other: Column): Column - def equalTo(other: Any): Column - def <=> (other: Column): Column - def <=> (other: Any): Column - def !== (other: Column): Column - def !== (other: Any): Column - - @scala.annotation.varargs - def in(list: Column*): Column - - def like(other: String): Column - def rlike(other: String): Column - - def contains(other: Column): Column - def contains(other: Any): Column - def startsWith(other: Column): Column - def startsWith(other: String): Column - def endsWith(other: Column): Column - def endsWith(other: String): Column - - def substr(startPos: Column, len: Column): Column - def substr(startPos: Int, len: Int): Column - - def isNull: Column - def isNotNull: Column - - def getItem(ordinal: Int): Column - def getField(fieldName: String): Column - - def cast(to: DataType): Column - def cast(to: String): Column - - def asc: Column - def desc: Column - - def as(alias: String): Column -} - - -/** - * An internal interface defining aggregation APIs for [[DataFrame]]. - * Please use [[DataFrame]] and [[GroupedDataFrame]] directly, and do NOT use this. - */ -private[sql] trait GroupedDataFrameApi { - - def agg(exprs: Map[String, String]): DataFrame - - @scala.annotation.varargs - def agg(expr: Column, exprs: Column*): DataFrame - - def avg(): DataFrame - - def mean(): DataFrame - - def min(): DataFrame - - def max(): DataFrame - - def sum(): DataFrame - - def count(): DataFrame - - // TODO: Add var, std -} From 523a93523d0f9fc12de1ba2dc1acc360cdbc7027 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 3 Feb 2015 00:29:23 -0800 Subject: [PATCH 041/578] [SPARK-5551][SQL] Create type alias for SchemaRDD for source backward compatibility Author: Reynold Xin Closes #4327 from rxin/schemarddTypeAlias and squashes the following commits: e5a8ff3 [Reynold Xin] [SPARK-5551][SQL] Create type alias for SchemaRDD for source backward compatibility --- sql/core/src/main/scala/org/apache/spark/sql/package.scala | 6 ++++++ 1 file changed, 6 insertions(+) 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 7c49b5220d607..02e5b015e8ec2 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 @@ -38,4 +38,10 @@ package object sql { */ @DeveloperApi protected[sql] type Strategy = org.apache.spark.sql.catalyst.planning.GenericStrategy[SparkPlan] + + /** + * Type alias for [[DataFrame]]. Kept here for backward source compatibility for Scala. + */ + @deprecated("1.3.0", "use DataFrame") + type SchemaRDD = DataFrame } From bebf4c42bef3e75d31ffce9bfdb331c16f34ddb1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 3 Feb 2015 00:46:04 -0800 Subject: [PATCH 042/578] [SPARK-5549] Define TaskContext interface in Scala. So the interface documentation shows up in ScalaDoc. Author: Reynold Xin Closes #4324 from rxin/TaskContext-scala and squashes the following commits: 2480a17 [Reynold Xin] comment 573756f [Reynold Xin] style fixes and javadoc fixes. 87dd537 [Reynold Xin] [SPARK-5549] Define TaskContext interface in Scala. --- .../java/org/apache/spark/TaskContext.java | 126 ---------------- .../scala/org/apache/spark/TaskContext.scala | 136 ++++++++++++++++++ .../org/apache/spark/TaskContextImpl.scala | 8 +- .../JavaTaskCompletionListenerImpl.java | 3 +- .../spark/JavaTaskContextCompileCheck.java | 41 ++++++ 5 files changed, 183 insertions(+), 131 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/TaskContext.java create mode 100644 core/src/main/scala/org/apache/spark/TaskContext.scala rename core/src/test/java/{org/apache/spark/util => test/org/apache/spark}/JavaTaskCompletionListenerImpl.java (93%) create mode 100644 core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java diff --git a/core/src/main/java/org/apache/spark/TaskContext.java b/core/src/main/java/org/apache/spark/TaskContext.java deleted file mode 100644 index 095f9fb94fdf0..0000000000000 --- a/core/src/main/java/org/apache/spark/TaskContext.java +++ /dev/null @@ -1,126 +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; - -import java.io.Serializable; - -import scala.Function0; -import scala.Function1; -import scala.Unit; - -import org.apache.spark.annotation.DeveloperApi; -import org.apache.spark.executor.TaskMetrics; -import org.apache.spark.util.TaskCompletionListener; - -/** - * Contextual information about a task which can be read or mutated during - * execution. To access the TaskContext for a running task use - * TaskContext.get(). - */ -public abstract class TaskContext implements Serializable { - /** - * Return the currently active TaskContext. This can be called inside of - * user functions to access contextual information about running tasks. - */ - public static TaskContext get() { - return taskContext.get(); - } - - private static ThreadLocal taskContext = - new ThreadLocal(); - - static void setTaskContext(TaskContext tc) { - taskContext.set(tc); - } - - static void unset() { - taskContext.remove(); - } - - /** - * Whether the task has completed. - */ - public abstract boolean isCompleted(); - - /** - * Whether the task has been killed. - */ - public abstract boolean isInterrupted(); - - /** @deprecated use {@link #isRunningLocally()} */ - @Deprecated - public abstract boolean runningLocally(); - - public abstract boolean isRunningLocally(); - - /** - * Add a (Java friendly) listener to be executed on task completion. - * This will be called in all situation - success, failure, or cancellation. - * An example use is for HadoopRDD to register a callback to close the input stream. - */ - public abstract TaskContext addTaskCompletionListener(TaskCompletionListener listener); - - /** - * Add a listener in the form of a Scala closure to be executed on task completion. - * This will be called in all situations - success, failure, or cancellation. - * An example use is for HadoopRDD to register a callback to close the input stream. - */ - public abstract TaskContext addTaskCompletionListener(final Function1 f); - - /** - * Add a callback function to be executed on task completion. An example use - * is for HadoopRDD to register a callback to close the input stream. - * Will be called in any situation - success, failure, or cancellation. - * - * @deprecated use {@link #addTaskCompletionListener(scala.Function1)} - * - * @param f Callback function. - */ - @Deprecated - public abstract void addOnCompleteCallback(final Function0 f); - - /** - * The ID of the stage that this task belong to. - */ - public abstract int stageId(); - - /** - * The ID of the RDD partition that is computed by this task. - */ - public abstract int partitionId(); - - /** - * How many times this task has been attempted. The first task attempt will be assigned - * attemptNumber = 0, and subsequent attempts will have increasing attempt numbers. - */ - public abstract int attemptNumber(); - - /** @deprecated use {@link #taskAttemptId()}; it was renamed to avoid ambiguity. */ - @Deprecated - public abstract long attemptId(); - - /** - * An ID that is unique to this task attempt (within the same SparkContext, no two task attempts - * will share the same attempt ID). This is roughly equivalent to Hadoop's TaskAttemptID. - */ - public abstract long taskAttemptId(); - - /** ::DeveloperApi:: */ - @DeveloperApi - public abstract TaskMetrics taskMetrics(); -} diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala new file mode 100644 index 0000000000000..af9c138f97877 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import java.io.Serializable + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.util.TaskCompletionListener + + +object TaskContext { + /** + * Return the currently active TaskContext. This can be called inside of + * user functions to access contextual information about running tasks. + */ + def get(): TaskContext = taskContext.get + + private val taskContext: ThreadLocal[TaskContext] = new ThreadLocal[TaskContext] + + // Note: protected[spark] instead of private[spark] to prevent the following two from + // showing up in JavaDoc. + /** + * Set the thread local TaskContext. Internal to Spark. + */ + protected[spark] def setTaskContext(tc: TaskContext): Unit = taskContext.set(tc) + + /** + * Unset the thread local TaskContext. Internal to Spark. + */ + protected[spark] def unset(): Unit = taskContext.remove() +} + + +/** + * Contextual information about a task which can be read or mutated during + * execution. To access the TaskContext for a running task, use: + * {{{ + * org.apache.spark.TaskContext.get() + * }}} + */ +abstract class TaskContext extends Serializable { + // Note: TaskContext must NOT define a get method. Otherwise it will prevent the Scala compiler + // from generating a static get method (based on the companion object's get method). + + // Note: Update JavaTaskContextCompileCheck when new methods are added to this class. + + // Note: getters in this class are defined with parentheses to maintain backward compatibility. + + /** + * Returns true if the task has completed. + */ + def isCompleted(): Boolean + + /** + * Returns true if the task has been killed. + */ + def isInterrupted(): Boolean + + @deprecated("1.2.0", "use isRunningLocally") + def runningLocally(): Boolean + + /** + * Returns true if the task is running locally in the driver program. + * @return + */ + def isRunningLocally(): Boolean + + /** + * Adds a (Java friendly) listener to be executed on task completion. + * This will be called in all situation - success, failure, or cancellation. + * An example use is for HadoopRDD to register a callback to close the input stream. + */ + def addTaskCompletionListener(listener: TaskCompletionListener): TaskContext + + /** + * Adds a listener in the form of a Scala closure to be executed on task completion. + * This will be called in all situations - success, failure, or cancellation. + * An example use is for HadoopRDD to register a callback to close the input stream. + */ + def addTaskCompletionListener(f: (TaskContext) => Unit): TaskContext + + /** + * Adds a callback function to be executed on task completion. An example use + * is for HadoopRDD to register a callback to close the input stream. + * Will be called in any situation - success, failure, or cancellation. + * + * @param f Callback function. + */ + @deprecated("1.2.0", "use addTaskCompletionListener") + def addOnCompleteCallback(f: () => Unit) + + /** + * The ID of the stage that this task belong to. + */ + def stageId(): Int + + /** + * The ID of the RDD partition that is computed by this task. + */ + def partitionId(): Int + + /** + * How many times this task has been attempted. The first task attempt will be assigned + * attemptNumber = 0, and subsequent attempts will have increasing attempt numbers. + */ + def attemptNumber(): Int + + @deprecated("1.3.0", "use attemptNumber") + def attemptId(): Long + + /** + * An ID that is unique to this task attempt (within the same SparkContext, no two task attempts + * will share the same attempt ID). This is roughly equivalent to Hadoop's TaskAttemptID. + */ + def taskAttemptId(): Long + + /** ::DeveloperApi:: */ + @DeveloperApi + def taskMetrics(): TaskMetrics +} diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 9bb0c61e441f8..337c8e4ebebcd 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -33,7 +33,7 @@ private[spark] class TaskContextImpl( with Logging { // For backwards-compatibility; this method is now deprecated as of 1.3.0. - override def attemptId: Long = taskAttemptId + override def attemptId(): Long = taskAttemptId // List of callback functions to execute when the task completes. @transient private val onCompleteCallbacks = new ArrayBuffer[TaskCompletionListener] @@ -87,10 +87,10 @@ private[spark] class TaskContextImpl( interrupted = true } - override def isCompleted: Boolean = completed + override def isCompleted(): Boolean = completed - override def isRunningLocally: Boolean = runningLocally + override def isRunningLocally(): Boolean = runningLocally - override def isInterrupted: Boolean = interrupted + override def isInterrupted(): Boolean = interrupted } diff --git a/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java b/core/src/test/java/test/org/apache/spark/JavaTaskCompletionListenerImpl.java similarity index 93% rename from core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java rename to core/src/test/java/test/org/apache/spark/JavaTaskCompletionListenerImpl.java index e9ec700e32e15..e38bc38949d7c 100644 --- a/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java +++ b/core/src/test/java/test/org/apache/spark/JavaTaskCompletionListenerImpl.java @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.spark.util; +package test.org.apache.spark; import org.apache.spark.TaskContext; +import org.apache.spark.util.TaskCompletionListener; /** diff --git a/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java b/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java new file mode 100644 index 0000000000000..4a918f725dc91 --- /dev/null +++ b/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark; + +import org.apache.spark.TaskContext; + +/** + * Something to make sure that TaskContext can be used in Java. + */ +public class JavaTaskContextCompileCheck { + + public static void test() { + TaskContext tc = TaskContext.get(); + + tc.isCompleted(); + tc.isInterrupted(); + tc.isRunningLocally(); + + tc.addTaskCompletionListener(new JavaTaskCompletionListenerImpl()); + + tc.attemptNumber(); + tc.partitionId(); + tc.stageId(); + tc.taskAttemptId(); + } +} From f7948f3f5718b7c4a2d35634815670c4cbbe70fd Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 3 Feb 2015 10:34:16 -0800 Subject: [PATCH 043/578] Minor: Fix TaskContext deprecated annotations. Made a mistake in https://github.com/apache/spark/pull/4324 Author: Reynold Xin Closes #4333 from rxin/taskcontext-deprecate and squashes the following commits: 61c44ee [Reynold Xin] Minor: Fix TaskContext deprecated annotations. --- core/src/main/scala/org/apache/spark/TaskContext.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index af9c138f97877..7d7fe1a446313 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -72,7 +72,7 @@ abstract class TaskContext extends Serializable { */ def isInterrupted(): Boolean - @deprecated("1.2.0", "use isRunningLocally") + @deprecated("use isRunningLocally", "1.2.0") def runningLocally(): Boolean /** @@ -102,7 +102,7 @@ abstract class TaskContext extends Serializable { * * @param f Callback function. */ - @deprecated("1.2.0", "use addTaskCompletionListener") + @deprecated("use addTaskCompletionListener", "1.2.0") def addOnCompleteCallback(f: () => Unit) /** @@ -121,7 +121,7 @@ abstract class TaskContext extends Serializable { */ def attemptNumber(): Int - @deprecated("1.3.0", "use attemptNumber") + @deprecated("use attemptNumber", "1.3.0") def attemptId(): Long /** From 4204a1271d5bff4dd64f46eed9ee80b30081f9dc Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 3 Feb 2015 10:34:56 -0800 Subject: [PATCH 044/578] [SQL] DataFrame API update 1. Added Java-friendly version of the expression operators (i.e. gt, geq) 2. Added JavaDoc for most operators 3. Simplified expression operators by having only one version of the function (that accepts Any). Previously we had two methods for each expression operator, one accepting Any and another accepting Column. 4. agg function now accepts varargs of (String, String). Author: Reynold Xin Closes #4332 from rxin/df-update and squashes the following commits: ab0aa69 [Reynold Xin] Added Java friendly expression methods. Added JavaDoc. For each expression operator, have only one version of the function (that accepts Any). Previously we had two methods for each expression operator, one accepting Any and another accepting Column. 576d07a [Reynold Xin] random commit. --- .../scala/org/apache/spark/sql/Column.scala | 439 ++++++++++-------- .../org/apache/spark/sql/DataFrame.scala | 45 +- .../org/apache/spark/sql/DataFrameImpl.scala | 19 +- .../main/scala/org/apache/spark/sql/Dsl.scala | 87 +++- .../apache/spark/sql/GroupedDataFrame.scala | 46 +- .../apache/spark/sql/IncomputableColumn.scala | 10 +- 6 files changed, 376 insertions(+), 270 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 0d6055ff232b1..4aa37219e13a6 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 @@ -65,7 +65,15 @@ trait Column extends DataFrame { */ def isComputable: Boolean - private def constructColumn(other: Column)(newExpr: Expression): Column = { + private def computableCol(baseCol: ComputableColumn, expr: Expression) = { + val plan = Project(Seq(expr match { + case named: NamedExpression => named + case unnamed: Expression => Alias(unnamed, "col")() + }), baseCol.plan) + Column(baseCol.sqlContext, plan, expr) + } + + 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) @@ -73,392 +81,423 @@ trait Column extends DataFrame { case _ => p } - def computableCol(baseCol: ComputableColumn, expr: Expression) = { - val plan = Project(Seq(expr match { - case named: NamedExpression => named - case unnamed: Expression => Alias(unnamed, "col")() - }), baseCol.plan) - Column(baseCol.sqlContext, plan, expr) - } - - (this, other) match { + (this, lit(otherValue)) match { case (left: ComputableColumn, right: ComputableColumn) => if (stripProject(left.plan).sameResult(stripProject(right.plan))) { - computableCol(right, newExpr) + computableCol(right, newExpr(right)) } else { - Column(newExpr) + Column(newExpr(right)) } - case (left: ComputableColumn, _) => computableCol(left, newExpr) - case (_, right: ComputableColumn) => computableCol(right, newExpr) - case (_, _) => Column(newExpr) + case (left: ComputableColumn, right) => computableCol(left, newExpr(right)) + case (_, right: ComputableColumn) => computableCol(right, newExpr(right)) + case (_, right) => Column(newExpr(right)) + } + } + + /** 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) } } /** * Unary minus, i.e. negate the expression. * {{{ - * // Select the amount column and negates all values. + * // Scala: select the amount column and negates all values. * df.select( -df("amount") ) + * + * // Java: + * import static org.apache.spark.sql.Dsl.*; + * df.select( negate(col("amount") ); * }}} */ - def unary_- : Column = constructColumn(null) { UnaryMinus(expr) } + def unary_- : Column = exprToColumn(UnaryMinus(expr)) /** * Bitwise NOT. * {{{ - * // Select the flags column and negate every bit. + * // Scala: select the flags column and negate every bit. * df.select( ~df("flags") ) * }}} */ - def unary_~ : Column = constructColumn(null) { BitwiseNot(expr) } + def unary_~ : Column = exprToColumn(BitwiseNot(expr)) /** * Inversion of boolean expression, i.e. NOT. * {{ - * // Select rows that are not active (isActive === false) - * df.select( !df("isActive") ) + * // Scala: select rows that are not active (isActive === false) + * df.filter( !df("isActive") ) + * + * // Java: + * import static org.apache.spark.sql.Dsl.*; + * df.filter( not(df.col("isActive")) ); * }} */ - def unary_! : Column = constructColumn(null) { Not(expr) } + def unary_! : Column = exprToColumn(Not(expr)) /** - * Equality test with an expression. + * Equality test. * {{{ - * // The following two both select rows in which colA equals colB. - * df.select( df("colA") === df("colB") ) - * df.select( df("colA".equalTo(df("colB")) ) + * // Scala: + * df.filter( df("colA") === df("colB") ) + * + * // Java + * import static org.apache.spark.sql.Dsl.*; + * df.filter( col("colA").equalTo(col("colB")) ); * }}} */ - def === (other: Column): Column = constructColumn(other) { - EqualTo(expr, other.expr) + def === (other: Any): Column = constructColumn(other) { o => + EqualTo(expr, o.expr) } /** - * Equality test with a literal value. - * {{{ - * // The following two both select rows in which colA is "Zaharia". - * df.select( df("colA") === "Zaharia") - * df.select( df("colA".equalTo("Zaharia") ) - * }}} - */ - def === (literal: Any): Column = this === lit(literal) - - /** - * Equality test with an expression. - * {{{ - * // The following two both select rows in which colA equals colB. - * df.select( df("colA") === df("colB") ) - * df.select( df("colA".equalTo(df("colB")) ) - * }}} - */ - def equalTo(other: Column): Column = this === other - - /** - * Equality test with a literal value. + * Equality test. * {{{ - * // The following two both select rows in which colA is "Zaharia". - * df.select( df("colA") === "Zaharia") - * df.select( df("colA".equalTo("Zaharia") ) + * // Scala: + * df.filter( df("colA") === df("colB") ) + * + * // Java + * import static org.apache.spark.sql.Dsl.*; + * df.filter( col("colA").equalTo(col("colB")) ); * }}} */ - def equalTo(literal: Any): Column = this === literal + def equalTo(other: Any): Column = this === other /** - * Inequality test with an expression. + * Inequality test. * {{{ - * // The following two both select rows in which colA does not equal colB. + * // Scala: * df.select( df("colA") !== df("colB") ) * df.select( !(df("colA") === df("colB")) ) + * + * // Java: + * import static org.apache.spark.sql.Dsl.*; + * df.filter( not(col("colA").equalTo(col("colB"))) ); * }}} */ - def !== (other: Column): Column = constructColumn(other) { - Not(EqualTo(expr, other.expr)) + def !== (other: Any): Column = constructColumn(other) { o => + Not(EqualTo(expr, o.expr)) } /** - * Inequality test with a literal value. - * {{{ - * // The following two both select rows in which colA does not equal equal 15. - * df.select( df("colA") !== 15 ) - * df.select( !(df("colA") === 15) ) - * }}} - */ - def !== (literal: Any): Column = this !== lit(literal) - - /** - * Greater than an expression. + * Greater than. * {{{ - * // The following selects people older than 21. - * people.select( people("age") > Literal(21) ) + * // Scala: The following selects people older than 21. + * people.select( people("age") > 21 ) + * + * // Java: + * import static org.apache.spark.sql.Dsl.*; + * people.select( people("age").gt(21) ); * }}} */ - def > (other: Column): Column = constructColumn(other) { - GreaterThan(expr, other.expr) + def > (other: Any): Column = constructColumn(other) { o => + GreaterThan(expr, o.expr) } /** - * Greater than a literal value. + * Greater than. * {{{ - * // The following selects people older than 21. - * people.select( people("age") > 21 ) + * // Scala: The following selects people older than 21. + * people.select( people("age") > lit(21) ) + * + * // Java: + * import static org.apache.spark.sql.Dsl.*; + * people.select( people("age").gt(21) ); * }}} */ - def > (literal: Any): Column = this > lit(literal) + def gt(other: Any): Column = this > other /** - * Less than an expression. + * Less than. * {{{ - * // The following selects people younger than 21. - * people.select( people("age") < Literal(21) ) + * // Scala: The following selects people younger than 21. + * people.select( people("age") < 21 ) + * + * // Java: + * people.select( people("age").lt(21) ); * }}} */ - def < (other: Column): Column = constructColumn(other) { - LessThan(expr, other.expr) + def < (other: Any): Column = constructColumn(other) { o => + LessThan(expr, o.expr) } /** - * Less than a literal value. + * Less than. * {{{ - * // The following selects people younger than 21. + * // Scala: The following selects people younger than 21. * people.select( people("age") < 21 ) + * + * // Java: + * people.select( people("age").lt(21) ); * }}} */ - def < (literal: Any): Column = this < lit(literal) + def lt(other: Any): Column = this < other /** - * Less than or equal to an expression. + * Less than or equal to. * {{{ - * // The following selects people age 21 or younger than 21. - * people.select( people("age") <= Literal(21) ) + * // Scala: The following selects people age 21 or younger than 21. + * people.select( people("age") <= 21 ) + * + * // Java: + * people.select( people("age").leq(21) ); * }}} */ - def <= (other: Column): Column = constructColumn(other) { - LessThanOrEqual(expr, other.expr) + def <= (other: Any): Column = constructColumn(other) { o => + LessThanOrEqual(expr, o.expr) } /** - * Less than or equal to a literal value. + * Less than or equal to. * {{{ - * // The following selects people age 21 or younger than 21. + * // Scala: The following selects people age 21 or younger than 21. * people.select( people("age") <= 21 ) + * + * // Java: + * people.select( people("age").leq(21) ); * }}} */ - def <= (literal: Any): Column = this <= lit(literal) + def leq(other: Any): Column = this <= other /** * Greater than or equal to an expression. * {{{ - * // The following selects people age 21 or older than 21. - * people.select( people("age") >= Literal(21) ) + * // Scala: The following selects people age 21 or older than 21. + * people.select( people("age") >= 21 ) + * + * // Java: + * people.select( people("age").geq(21) ) * }}} */ - def >= (other: Column): Column = constructColumn(other) { - GreaterThanOrEqual(expr, other.expr) + def >= (other: Any): Column = constructColumn(other) { o => + GreaterThanOrEqual(expr, o.expr) } /** - * Greater than or equal to a literal value. + * Greater than or equal to an expression. * {{{ - * // The following selects people age 21 or older than 21. + * // Scala: The following selects people age 21 or older than 21. * people.select( people("age") >= 21 ) + * + * // Java: + * people.select( people("age").geq(21) ) * }}} */ - def >= (literal: Any): Column = this >= lit(literal) + def geq(other: Any): Column = this >= other /** - * Equality test with an expression that is safe for null values. + * Equality test that is safe for null values. */ - def <=> (other: Column): Column = constructColumn(other) { - other match { - case null => EqualNullSafe(expr, lit(null).expr) - case _ => EqualNullSafe(expr, other.expr) - } + def <=> (other: Any): Column = constructColumn(other) { o => + EqualNullSafe(expr, o.expr) } /** - * Equality test with a literal value that is safe for null values. + * Equality test that is safe for null values. */ - def <=> (literal: Any): Column = this <=> lit(literal) + def eqNullSafe(other: Any): Column = this <=> other /** * True if the current expression is null. */ - def isNull: Column = constructColumn(null) { IsNull(expr) } + def isNull: Column = exprToColumn(IsNull(expr)) /** * True if the current expression is NOT null. */ - def isNotNull: Column = constructColumn(null) { IsNotNull(expr) } + def isNotNull: Column = exprToColumn(IsNotNull(expr)) /** - * Boolean OR with an expression. + * Boolean OR. * {{{ - * // The following selects people that are in school or employed. - * people.select( people("inSchool") || people("isEmployed") ) + * // Scala: The following selects people that are in school or employed. + * people.filter( people("inSchool") || people("isEmployed") ) + * + * // Java: + * people.filter( people("inSchool").or(people("isEmployed")) ); * }}} */ - def || (other: Column): Column = constructColumn(other) { - Or(expr, other.expr) + def || (other: Any): Column = constructColumn(other) { o => + Or(expr, o.expr) } /** - * Boolean OR with a literal value. + * Boolean OR. * {{{ - * // The following selects everything. - * people.select( people("inSchool") || true ) + * // Scala: The following selects people that are in school or employed. + * people.filter( people("inSchool") || people("isEmployed") ) + * + * // Java: + * people.filter( people("inSchool").or(people("isEmployed")) ); * }}} */ - def || (literal: Boolean): Column = this || lit(literal) + def or(other: Column): Column = this || other /** - * Boolean AND with an expression. + * Boolean AND. * {{{ - * // The following selects people that are in school and employed at the same time. + * // Scala: The following selects people that are in school and employed at the same time. * people.select( people("inSchool") && people("isEmployed") ) + * + * // Java: + * people.select( people("inSchool").and(people("isEmployed")) ); * }}} */ - def && (other: Column): Column = constructColumn(other) { - And(expr, other.expr) + def && (other: Any): Column = constructColumn(other) { o => + And(expr, o.expr) } /** - * Boolean AND with a literal value. + * Boolean AND. * {{{ - * // The following selects people that are in school. - * people.select( people("inSchool") && true ) + * // Scala: The following selects people that are in school and employed at the same time. + * people.select( people("inSchool") && people("isEmployed") ) + * + * // Java: + * people.select( people("inSchool").and(people("isEmployed")) ); * }}} */ - def && (literal: Boolean): Column = this && lit(literal) + def and(other: Column): Column = this && other /** - * Bitwise AND with an expression. + * Bitwise AND. */ - def & (other: Column): Column = constructColumn(other) { - BitwiseAnd(expr, other.expr) + def & (other: Any): Column = constructColumn(other) { o => + BitwiseAnd(expr, o.expr) } - /** - * Bitwise AND with a literal value. - */ - def & (literal: Any): Column = this & lit(literal) - /** * Bitwise OR with an expression. */ - def | (other: Column): Column = constructColumn(other) { - BitwiseOr(expr, other.expr) + def | (other: Any): Column = constructColumn(other) { o => + BitwiseOr(expr, o.expr) } - /** - * Bitwise OR with a literal value. - */ - def | (literal: Any): Column = this | lit(literal) - /** * Bitwise XOR with an expression. */ - def ^ (other: Column): Column = constructColumn(other) { - BitwiseXor(expr, other.expr) + def ^ (other: Any): Column = constructColumn(other) { o => + BitwiseXor(expr, o.expr) } - /** - * Bitwise XOR with a literal value. - */ - def ^ (literal: Any): Column = this ^ lit(literal) - /** * Sum of this expression and another expression. * {{{ - * // The following selects the sum of a person's height and weight. + * // Scala: The following selects the sum of a person's height and weight. * people.select( people("height") + people("weight") ) + * + * // Java: + * people.select( people("height").plus(people("weight")) ); * }}} */ - def + (other: Column): Column = constructColumn(other) { - Add(expr, other.expr) + def + (other: Any): Column = constructColumn(other) { o => + Add(expr, o.expr) } /** * Sum of this expression and another expression. * {{{ - * // The following selects the sum of a person's height and 10. - * people.select( people("height") + 10 ) + * // Scala: The following selects the sum of a person's height and weight. + * people.select( people("height") + people("weight") ) + * + * // Java: + * people.select( people("height").plus(people("weight")) ); * }}} */ - def + (literal: Any): Column = this + lit(literal) + def plus(other: Any): Column = this + other /** * Subtraction. Subtract the other expression from this expression. * {{{ - * // The following selects the difference between people's height and their weight. + * // Scala: The following selects the difference between people's height and their weight. * people.select( people("height") - people("weight") ) + * + * // Java: + * people.select( people("height").minus(people("weight")) ); * }}} */ - def - (other: Column): Column = constructColumn(other) { - Subtract(expr, other.expr) + def - (other: Any): Column = constructColumn(other) { o => + Subtract(expr, o.expr) } /** - * Subtraction. Subtract a literal value from this expression. + * Subtraction. Subtract the other expression from this expression. * {{{ - * // The following selects a person's height and subtract it by 10. - * people.select( people("height") - 10 ) + * // Scala: The following selects the difference between people's height and their weight. + * people.select( people("height") - people("weight") ) + * + * // Java: + * people.select( people("height").minus(people("weight")) ); * }}} */ - def - (literal: Any): Column = this - lit(literal) + def minus(other: Any): Column = this - other /** * Multiplication of this expression and another expression. * {{{ - * // The following multiplies a person's height by their weight. + * // Scala: The following multiplies a person's height by their weight. * people.select( people("height") * people("weight") ) + * + * // Java: + * people.select( people("height").multiply(people("weight")) ); * }}} */ - def * (other: Column): Column = constructColumn(other) { - Multiply(expr, other.expr) + def * (other: Any): Column = constructColumn(other) { o => + Multiply(expr, o.expr) } /** - * Multiplication this expression and a literal value. + * Multiplication of this expression and another expression. * {{{ - * // The following multiplies a person's height by 10. - * people.select( people("height") * 10 ) + * // Scala: The following multiplies a person's height by their weight. + * people.select( people("height") * people("weight") ) + * + * // Java: + * people.select( people("height").multiply(people("weight")) ); * }}} */ - def * (literal: Any): Column = this * lit(literal) + def multiply(other: Any): Column = this * other /** * Division this expression by another expression. * {{{ - * // The following divides a person's height by their weight. + * // Scala: The following divides a person's height by their weight. * people.select( people("height") / people("weight") ) + * + * // Java: + * people.select( people("height").divide(people("weight")) ); * }}} */ - def / (other: Column): Column = constructColumn(other) { - Divide(expr, other.expr) + def / (other: Any): Column = constructColumn(other) { o => + Divide(expr, o.expr) } /** - * Division this expression by a literal value. + * Division this expression by another expression. * {{{ - * // The following divides a person's height by 10. - * people.select( people("height") / 10 ) + * // Scala: The following divides a person's height by their weight. + * people.select( people("height") / people("weight") ) + * + * // Java: + * people.select( people("height").divide(people("weight")) ); * }}} */ - def / (literal: Any): Column = this / lit(literal) + def divide(other: Any): Column = this / other /** * Modulo (a.k.a. remainder) expression. */ - def % (other: Column): Column = constructColumn(other) { - Remainder(expr, other.expr) + def % (other: Any): Column = constructColumn(other) { o => + Remainder(expr, o.expr) } /** * Modulo (a.k.a. remainder) expression. */ - def % (literal: Any): Column = this % lit(literal) - + def mod(other: Any): Column = this % other /** * A boolean expression that is evaluated to true if the value of this expression is contained @@ -469,27 +508,19 @@ trait Column extends DataFrame { new IncomputableColumn(In(expr, list.map(_.expr))) } - def like(literal: String): Column = constructColumn(null) { - Like(expr, lit(literal).expr) - } + def like(literal: String): Column = exprToColumn(Like(expr, lit(literal).expr)) - def rlike(literal: String): Column = constructColumn(null) { - RLike(expr, lit(literal).expr) - } + def rlike(literal: String): Column = exprToColumn(RLike(expr, lit(literal).expr)) /** * An expression that gets an item at position `ordinal` out of an array. */ - def getItem(ordinal: Int): Column = constructColumn(null) { - GetItem(expr, Literal(ordinal)) - } + def getItem(ordinal: Int): Column = exprToColumn(GetItem(expr, Literal(ordinal))) /** * An expression that gets a field by name in a [[StructField]]. */ - def getField(fieldName: String): Column = constructColumn(null) { - GetField(expr, fieldName) - } + def getField(fieldName: String): Column = exprToColumn(GetField(expr, fieldName)) /** * An expression that returns a substring. @@ -507,20 +538,18 @@ trait Column extends DataFrame { */ def substr(startPos: Int, len: Int): Column = this.substr(lit(startPos), lit(len)) - def contains(other: Column): Column = constructColumn(other) { - Contains(expr, other.expr) + def contains(other: Any): Column = constructColumn(other) { o => + Contains(expr, o.expr) } - def contains(literal: Any): Column = this.contains(lit(literal)) - - def startsWith(other: Column): Column = constructColumn(other) { - StartsWith(expr, other.expr) + def startsWith(other: Column): Column = constructColumn(other) { o => + StartsWith(expr, o.expr) } def startsWith(literal: String): Column = this.startsWith(lit(literal)) - def endsWith(other: Column): Column = constructColumn(other) { - EndsWith(expr, other.expr) + def endsWith(other: Column): Column = constructColumn(other) { o => + EndsWith(expr, o.expr) } def endsWith(literal: String): Column = this.endsWith(lit(literal)) @@ -532,7 +561,7 @@ trait Column extends DataFrame { * df.select($"colA".as("colB")) * }}} */ - override def as(alias: String): Column = constructColumn(null) { Alias(expr, alias)() } + override def as(alias: String): Column = exprToColumn(Alias(expr, alias)()) /** * Casts the column to a different data type. @@ -545,7 +574,7 @@ trait Column extends DataFrame { * df.select(df("colA").cast("int")) * }}} */ - def cast(to: DataType): Column = constructColumn(null) { Cast(expr, to) } + def cast(to: DataType): Column = exprToColumn(Cast(expr, to)) /** * Casts the column to a different data type, using the canonical string representation @@ -556,7 +585,7 @@ trait Column extends DataFrame { * df.select(df("colA").cast("int")) * }}} */ - def cast(to: String): Column = constructColumn(null) { + def cast(to: String): Column = exprToColumn( Cast(expr, to.toLowerCase match { case "string" => StringType case "boolean" => BooleanType @@ -571,11 +600,11 @@ trait Column extends DataFrame { case "timestamp" => TimestampType case _ => throw new RuntimeException(s"""Unsupported cast type: "$to"""") }) - } + ) - def desc: Column = constructColumn(null) { SortOrder(expr, Descending) } + def desc: Column = exprToColumn(SortOrder(expr, Descending), computable = false) - def asc: Column = constructColumn(null) { SortOrder(expr, Ascending) } + def asc: Column = exprToColumn(SortOrder(expr, Ascending), computable = 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 5920852e8c178..f3bc07ae5238c 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 @@ -25,6 +25,7 @@ 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.types.StructType +import org.apache.spark.util.Utils private[sql] object DataFrame { @@ -138,7 +139,13 @@ trait DataFrame extends RDDApi[Row] { * a full outer join between `df1` and `df2`. * * {{{ + * // Scala: + * import org.apache.spark.sql.dsl._ * df1.join(df2, "outer", $"df1Key" === $"df2Key") + * + * // Java: + * import static org.apache.spark.sql.Dsl.*; + * df1.join(df2, "outer", col("df1Key") === col("df2Key")); * }}} * * @param right Right side of the join. @@ -185,7 +192,12 @@ trait DataFrame extends RDDApi[Row] { /** * Selects column based on the column name and return it as a [[Column]]. */ - def apply(colName: String): Column + def apply(colName: String): Column = col(colName) + + /** + * Selects column based on the column name and return it as a [[Column]]. + */ + def col(colName: String): Column /** * Selects a set of expressions, wrapped in a Product. @@ -297,24 +309,41 @@ trait DataFrame extends RDDApi[Row] { def groupBy(col1: String, cols: String*): GroupedDataFrame /** - * Aggregates on the entire [[DataFrame]] without groups. + * (Scala-specific) Compute aggregates by specifying a map from column name to + * aggregate methods. The resulting [[DataFrame]] will also contain the grouping columns. + * + * The available aggregate methods are `avg`, `max`, `min`, `sum`, `count`. + * {{{ + * // Selects the age of the oldest employee and the aggregate expense for each department + * df.groupBy("department").agg( + * "age" -> "max", + * "expense" -> "sum" + * ) + * }}} + */ + def agg(aggExpr: (String, String), aggExprs: (String, String)*): DataFrame = { + groupBy().agg(aggExpr, aggExprs :_*) + } + + /** + * (Scala-specific) Aggregates on the entire [[DataFrame]] without groups. * {{ * // df.agg(...) is a shorthand for df.groupBy().agg(...) * df.agg(Map("age" -> "max", "salary" -> "avg")) * df.groupBy().agg(Map("age" -> "max", "salary" -> "avg")) * }} */ - def agg(exprs: Map[String, String]): DataFrame + def agg(exprs: Map[String, String]): DataFrame = groupBy().agg(exprs) /** - * Aggregates on the entire [[DataFrame]] without groups. + * (Java-specific) Aggregates on the entire [[DataFrame]] without groups. * {{ * // df.agg(...) is a shorthand for df.groupBy().agg(...) * df.agg(Map("age" -> "max", "salary" -> "avg")) * df.groupBy().agg(Map("age" -> "max", "salary" -> "avg")) * }} */ - def agg(exprs: java.util.Map[String, String]): DataFrame + def agg(exprs: java.util.Map[String, String]): DataFrame = groupBy().agg(exprs) /** * Aggregates on the entire [[DataFrame]] without groups. @@ -325,7 +354,7 @@ trait DataFrame extends RDDApi[Row] { * }} */ @scala.annotation.varargs - def agg(expr: Column, exprs: Column*): DataFrame + def agg(expr: Column, exprs: Column*): DataFrame = groupBy().agg(expr, exprs :_*) /** * Returns a new [[DataFrame]] by taking the first `n` rows. The difference between this function @@ -366,7 +395,9 @@ trait DataFrame extends RDDApi[Row] { * @param withReplacement Sample with replacement or not. * @param fraction Fraction of rows to generate. */ - def sample(withReplacement: Boolean, fraction: Double): DataFrame + def sample(withReplacement: Boolean, fraction: Double): DataFrame = { + sample(withReplacement, fraction, Utils.random.nextLong) + } ///////////////////////////////////////////////////////////////////////////// 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 49fd131534bb5..0b0623dc1fe75 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 @@ -36,7 +36,6 @@ 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.types.{NumericType, StructType} -import org.apache.spark.util.Utils /** @@ -148,7 +147,7 @@ private[sql] class DataFrameImpl protected[sql]( sort(sortExpr, sortExprs :_*) } - override def apply(colName: String): Column = colName match { + override def col(colName: String): Column = colName match { case "*" => Column(ResolvedStar(schema.fieldNames.map(resolve))) case _ => @@ -201,18 +200,6 @@ private[sql] class DataFrameImpl protected[sql]( new GroupedDataFrame(this, colNames.map(colName => resolve(colName))) } - override def agg(exprs: Map[String, String]): DataFrame = { - groupBy().agg(exprs) - } - - override def agg(exprs: java.util.Map[String, String]): DataFrame = { - agg(exprs.toMap) - } - - override def agg(expr: Column, exprs: Column*): DataFrame = { - groupBy().agg(expr, exprs :_*) - } - override def limit(n: Int): DataFrame = { Limit(Literal(n), logicalPlan) } @@ -233,10 +220,6 @@ private[sql] class DataFrameImpl protected[sql]( Sample(fraction, withReplacement, seed, logicalPlan) } - override def sample(withReplacement: Boolean, fraction: Double): DataFrame = { - sample(withReplacement, fraction, Utils.random.nextLong) - } - ///////////////////////////////////////////////////////////////////////////// override def addColumn(colName: String, col: Column): 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 b4279a32ffa21..71365c776d559 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 @@ -36,21 +36,6 @@ object Dsl { /** An implicit conversion that turns a Scala `Symbol` into a [[Column]]. */ implicit def symbolToColumn(s: Symbol): ColumnName = new ColumnName(s.name) - // /** - // * An implicit conversion that turns a RDD of product into a [[DataFrame]]. - // * - // * This method requires an implicit SQLContext in scope. For example: - // * {{{ - // * implicit val sqlContext: SQLContext = ... - // * val rdd: RDD[(Int, String)] = ... - // * rdd.toDataFrame // triggers the implicit here - // * }}} - // */ - // implicit def rddToDataFrame[A <: Product: TypeTag](rdd: RDD[A])(implicit context: SQLContext) - // : DataFrame = { - // context.createDataFrame(rdd) - // } - /** Converts $"col name" into an [[Column]]. */ implicit class StringToColumn(val sc: StringContext) extends AnyVal { def $(args: Any*): ColumnName = { @@ -72,10 +57,16 @@ object Dsl { /** * Creates a [[Column]] of literal value. + * + * 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. */ def lit(literal: Any): Column = { - if (literal.isInstanceOf[Symbol]) { - return new ColumnName(literal.asInstanceOf[Symbol].name) + literal match { + case c: Column => return c + case s: Symbol => return new ColumnName(literal.asInstanceOf[Symbol].name) + case _ => // continue } val literalExpr = literal match { @@ -100,27 +91,82 @@ object Dsl { Column(literalExpr) } + ////////////////////////////////////////////////////////////////////////////////////////////// + ////////////////////////////////////////////////////////////////////////////////////////////// + + /** Aggregate function: returns the sum of all values in the expression. */ def sum(e: Column): Column = Sum(e.expr) + + /** Aggregate function: returns the sum of distinct values in the expression. */ def sumDistinct(e: Column): Column = SumDistinct(e.expr) + + /** Aggregate function: returns the number of items in a group. */ def count(e: Column): Column = Count(e.expr) + /** Aggregate function: returns the number of distinct items in a group. */ @scala.annotation.varargs def countDistinct(expr: Column, exprs: Column*): Column = CountDistinct((expr +: exprs).map(_.expr)) + /** Aggregate function: returns the approximate number of distinct items in a group. */ def approxCountDistinct(e: Column): Column = ApproxCountDistinct(e.expr) - def approxCountDistinct(e: Column, rsd: Double): Column = - ApproxCountDistinct(e.expr, rsd) + /** Aggregate function: returns the approximate number of distinct items in a group. */ + def approxCountDistinct(e: Column, rsd: Double): Column = ApproxCountDistinct(e.expr, rsd) + + /** Aggregate function: returns the average of the values in a group. */ def avg(e: Column): Column = Average(e.expr) + + /** Aggregate function: returns the first value in a group. */ def first(e: Column): Column = First(e.expr) + + /** Aggregate function: returns the last value in a group. */ def last(e: Column): Column = Last(e.expr) + + /** Aggregate function: returns the minimum value of the expression in a group. */ def min(e: Column): Column = Min(e.expr) + + /** Aggregate function: returns the maximum value of the expression in a group. */ def max(e: Column): Column = Max(e.expr) + ////////////////////////////////////////////////////////////////////////////////////////////// + ////////////////////////////////////////////////////////////////////////////////////////////// + + /** + * Unary minus, i.e. negate the expression. + * {{{ + * // Select the amount column and negates all values. + * // Scala: + * df.select( -df("amount") ) + * + * // Java: + * df.select( negate(df.col("amount")) ); + * }}} + */ + def negate(e: Column): Column = -e + + /** + * Inversion of boolean expression, i.e. NOT. + * {{ + * // Scala: select rows that are not active (isActive === false) + * df.filter( !df("isActive") ) + * + * // Java: + * df.filter( not(df.col("isActive")) ); + * }} + */ + def not(e: Column): Column = !e + + /** Converts a string expression to upper case. */ def upper(e: Column): Column = Upper(e.expr) + + /** Converts a string exprsesion to lower case. */ def lower(e: Column): Column = Lower(e.expr) + + /** Computes the square root of the specified float value. */ def sqrt(e: Column): Column = Sqrt(e.expr) + + /** Computes the absolutle value. */ def abs(e: Column): Column = Abs(e.expr) /** @@ -131,6 +177,9 @@ object Dsl { cols.toList.toSeq } + ////////////////////////////////////////////////////////////////////////////////////////////// + ////////////////////////////////////////////////////////////////////////////////////////////// + // scalastyle:off /* Use the following code to generate: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala index 6d0f3e8ce37cc..7963cb03126ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql -import java.util.{List => JList} - import scala.language.implicitConversions import scala.collection.JavaConversions._ @@ -59,15 +57,32 @@ class GroupedDataFrame protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expr } /** - * Compute aggregates by specifying a map from column name to aggregate methods. The resulting - * [[DataFrame]] will also contain the grouping columns. + * (Scala-specific) Compute aggregates by specifying a map from column name to + * aggregate methods. The resulting [[DataFrame]] will also contain the grouping columns. + * + * The available aggregate methods are `avg`, `max`, `min`, `sum`, `count`. + * {{{ + * // Selects the age of the oldest employee and the aggregate expense for each department + * df.groupBy("department").agg( + * "age" -> "max", + * "expense" -> "sum" + * ) + * }}} + */ + def agg(aggExpr: (String, String), aggExprs: (String, String)*): DataFrame = { + agg((aggExpr +: aggExprs).toMap) + } + + /** + * (Scala-specific) Compute aggregates by specifying a map from column name to + * aggregate methods. The resulting [[DataFrame]] will also contain the grouping columns. * * The available aggregate methods are `avg`, `max`, `min`, `sum`, `count`. * {{{ * // Selects the age of the oldest employee and the aggregate expense for each department * df.groupBy("department").agg(Map( - * "age" -> "max" - * "sum" -> "expense" + * "age" -> "max", + * "expense" -> "sum" * )) * }}} */ @@ -79,16 +94,17 @@ class GroupedDataFrame protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expr } /** - * Compute aggregates by specifying a map from column name to aggregate methods. The resulting - * [[DataFrame]] will also contain the grouping columns. + * (Java-specific) Compute aggregates by specifying a map from column name to + * aggregate methods. The resulting [[DataFrame]] will also contain the grouping columns. * * The available aggregate methods are `avg`, `max`, `min`, `sum`, `count`. * {{{ * // Selects the age of the oldest employee and the aggregate expense for each department - * df.groupBy("department").agg(Map( - * "age" -> "max" - * "sum" -> "expense" - * )) + * import com.google.common.collect.ImmutableMap; + * df.groupBy("department").agg(ImmutableMap.builder() + * .put("age", "max") + * .put("expense", "sum") + * .build()); * }}} */ def agg(exprs: java.util.Map[String, String]): DataFrame = { @@ -103,8 +119,14 @@ class GroupedDataFrame protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expr * * {{{ * // Selects the age of the oldest employee and the aggregate expense for each department + * + * // Scala: * import org.apache.spark.sql.dsl._ * df.groupBy("department").agg($"department", max($"age"), sum($"expense")) + * + * // Java: + * import static org.apache.spark.sql.Dsl.*; + * df.groupBy("department").agg(col("department"), max(col("age")), sum(col("expense"))); * }}} */ @scala.annotation.varargs 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 9b051de68feb8..ba5c7355b4b70 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 @@ -72,7 +72,7 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten override def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame = err() - override def apply(colName: String): Column = err() + override def col(colName: String): Column = err() override def apply(projection: Product): DataFrame = err() @@ -90,12 +90,6 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten override def groupBy(col1: String, cols: String*): GroupedDataFrame = err() - override def agg(exprs: Map[String, String]): DataFrame = err() - - override def agg(exprs: java.util.Map[String, String]): DataFrame = err() - - override def agg(expr: Column, exprs: Column*): DataFrame = err() - override def limit(n: Int): DataFrame = err() override def unionAll(other: DataFrame): DataFrame = err() @@ -106,8 +100,6 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten override def sample(withReplacement: Boolean, fraction: Double, seed: Long): DataFrame = err() - override def sample(withReplacement: Boolean, fraction: Double): DataFrame = err() - ///////////////////////////////////////////////////////////////////////////// override def addColumn(colName: String, col: Column): DataFrame = err() From 0c20ce69fb4bcb1cec5313a9d072826c5588cbbc Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Tue, 3 Feb 2015 12:06:06 -0800 Subject: [PATCH 045/578] [SPARK-4987] [SQL] parquet timestamp type support Author: Daoyuan Wang Closes #3820 from adrian-wang/parquettimestamp and squashes the following commits: b1e2a0d [Daoyuan Wang] fix for nanos 4dadef1 [Daoyuan Wang] fix wrong read 93f438d [Daoyuan Wang] parquet timestamp support --- docs/sql-programming-guide.md | 9 ++ pom.xml | 1 + sql/core/pom.xml | 5 + .../scala/org/apache/spark/sql/SQLConf.scala | 7 ++ .../spark/sql/parquet/ParquetConverter.scala | 94 ++++++++++++++++++- .../spark/sql/parquet/ParquetRelation.scala | 4 +- .../sql/parquet/ParquetTableSupport.scala | 13 ++- .../spark/sql/parquet/ParquetTestData.scala | 8 +- .../spark/sql/parquet/ParquetTypes.scala | 52 ++++++---- .../apache/spark/sql/parquet/newParquet.scala | 3 +- .../sql/parquet/timestamp/NanoTime.scala | 69 ++++++++++++++ 11 files changed, 239 insertions(+), 26 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/timestamp/NanoTime.scala diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index be8c5c2c1522e..22664b419f5cb 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -580,6 +580,15 @@ Configuration of Parquet can be done using the `setConf` method on SQLContext or flag tells Spark SQL to interpret binary data as a string to provide compatibility with these systems. + + + + + diff --git a/pom.xml b/pom.xml index e25eced877578..542efbaf06eb0 100644 --- a/pom.xml +++ b/pom.xml @@ -149,6 +149,7 @@ 2.10${scala.version}org.scala-lang + 3.6.31.8.81.1.1.6 diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 1a0c77d282307..03a5c9e7c24a0 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -66,6 +66,11 @@ jackson-databind2.3.0 + + org.jodd + jodd-core + ${jodd.version} + junit junit 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 561a91d2d60ee..7fe17944a734e 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 @@ -33,6 +33,7 @@ private[spark] object SQLConf { val DIALECT = "spark.sql.dialect" val PARQUET_BINARY_AS_STRING = "spark.sql.parquet.binaryAsString" + val PARQUET_INT96_AS_TIMESTAMP = "spark.sql.parquet.int96AsTimestamp" val PARQUET_CACHE_METADATA = "spark.sql.parquet.cacheMetadata" val PARQUET_COMPRESSION = "spark.sql.parquet.compression.codec" val PARQUET_FILTER_PUSHDOWN_ENABLED = "spark.sql.parquet.filterPushdown" @@ -143,6 +144,12 @@ private[sql] class SQLConf extends Serializable { private[spark] def isParquetBinaryAsString: Boolean = getConf(PARQUET_BINARY_AS_STRING, "false").toBoolean + /** + * When set to true, we always treat INT96Values in Parquet files as timestamp. + */ + private[spark] def isParquetINT96AsTimestamp: Boolean = + getConf(PARQUET_INT96_AS_TIMESTAMP, "true").toBoolean + /** * When set to true, partition pruning for in-memory columnar tables is enabled. */ 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 10df8c3310092..d87ddfeabda77 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 @@ -17,8 +17,12 @@ package org.apache.spark.sql.parquet +import java.sql.Timestamp +import java.util.{TimeZone, Calendar} + import scala.collection.mutable.{Buffer, ArrayBuffer, HashMap} +import jodd.datetime.JDateTime import parquet.column.Dictionary import parquet.io.api.{PrimitiveConverter, GroupConverter, Binary, Converter} import parquet.schema.MessageType @@ -26,6 +30,7 @@ import parquet.schema.MessageType import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.parquet.CatalystConverter.FieldType import org.apache.spark.sql.types._ +import org.apache.spark.sql.parquet.timestamp.NanoTime /** * Collection of converters of Parquet types (group and primitive types) that @@ -123,6 +128,12 @@ private[sql] object CatalystConverter { parent.updateDecimal(fieldIndex, value, d) } } + case TimestampType => { + new CatalystPrimitiveConverter(parent, fieldIndex) { + override def addBinary(value: Binary): Unit = + parent.updateTimestamp(fieldIndex, value) + } + } // All other primitive types use the default converter case ctype: PrimitiveType => { // note: need the type tag here! new CatalystPrimitiveConverter(parent, fieldIndex) @@ -197,9 +208,11 @@ private[parquet] abstract class CatalystConverter extends GroupConverter { protected[parquet] def updateString(fieldIndex: Int, value: String): Unit = updateField(fieldIndex, value) - protected[parquet] def updateDecimal(fieldIndex: Int, value: Binary, ctype: DecimalType): Unit = { + protected[parquet] def updateTimestamp(fieldIndex: Int, value: Binary): Unit = + updateField(fieldIndex, readTimestamp(value)) + + protected[parquet] def updateDecimal(fieldIndex: Int, value: Binary, ctype: DecimalType): Unit = updateField(fieldIndex, readDecimal(new Decimal(), value, ctype)) - } protected[parquet] def isRootConverter: Boolean = parent == null @@ -232,6 +245,13 @@ private[parquet] abstract class CatalystConverter extends GroupConverter { unscaled = (unscaled << (64 - numBits)) >> (64 - numBits) dest.set(unscaled, precision, scale) } + + /** + * Read a Timestamp value from a Parquet Int96Value + */ + protected[parquet] def readTimestamp(value: Binary): Timestamp = { + CatalystTimestampConverter.convertToTimestamp(value) + } } /** @@ -384,6 +404,9 @@ private[parquet] class CatalystPrimitiveRowConverter( override protected[parquet] def updateString(fieldIndex: Int, value: String): Unit = current.setString(fieldIndex, value) + override protected[parquet] def updateTimestamp(fieldIndex: Int, value: Binary): Unit = + current.update(fieldIndex, readTimestamp(value)) + override protected[parquet] def updateDecimal( fieldIndex: Int, value: Binary, ctype: DecimalType): Unit = { var decimal = current(fieldIndex).asInstanceOf[Decimal] @@ -454,6 +477,73 @@ private[parquet] object CatalystArrayConverter { val INITIAL_ARRAY_SIZE = 20 } +private[parquet] object CatalystTimestampConverter { + // TODO most part of this comes from Hive-0.14 + // Hive code might have some issues, so we need to keep an eye on it. + // 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 = { + // this is a cache for the calendar instance. + if (parquetTsCalendar.get == null) { + parquetTsCalendar.set(Calendar.getInstance(TimeZone.getTimeZone("GMT"))) + } + parquetTsCalendar.get + } + val NANOS_PER_SECOND: Long = 1000000000 + val SECONDS_PER_MINUTE: Long = 60 + val MINUTES_PER_HOUR: Long = 60 + val NANOS_PER_MILLI: Long = 1000000 + + def convertToTimestamp(value: Binary): Timestamp = { + val nt = NanoTime.fromBinary(value) + val timeOfDayNanos = nt.getTimeOfDayNanos + val julianDay = nt.getJulianDay + val jDateTime = new JDateTime(julianDay.toDouble) + val calendar = getCalendar + calendar.set(Calendar.YEAR, jDateTime.getYear) + calendar.set(Calendar.MONTH, jDateTime.getMonth - 1) + calendar.set(Calendar.DAY_OF_MONTH, jDateTime.getDay) + + // written in command style + var remainder = timeOfDayNanos + calendar.set( + Calendar.HOUR_OF_DAY, + (remainder / (NANOS_PER_SECOND * SECONDS_PER_MINUTE * MINUTES_PER_HOUR)).toInt) + remainder = remainder % (NANOS_PER_SECOND * SECONDS_PER_MINUTE * MINUTES_PER_HOUR) + calendar.set( + Calendar.MINUTE, (remainder / (NANOS_PER_SECOND * SECONDS_PER_MINUTE)).toInt) + remainder = remainder % (NANOS_PER_SECOND * SECONDS_PER_MINUTE) + calendar.set(Calendar.SECOND, (remainder / NANOS_PER_SECOND).toInt) + val nanos = remainder % NANOS_PER_SECOND + val ts = new Timestamp(calendar.getTimeInMillis) + ts.setNanos(nanos.toInt) + ts + } + + def convertFromTimestamp(ts: Timestamp): Binary = { + val calendar = getCalendar + calendar.setTime(ts) + val jDateTime = new JDateTime(calendar.get(Calendar.YEAR), + calendar.get(Calendar.MONTH) + 1, calendar.get(Calendar.DAY_OF_MONTH)) + // Hive-0.14 didn't set hour before get day number, while the day number should + // has something to do with hour, since julian day number grows at 12h GMT + // here we just follow what hive does. + val julianDay = jDateTime.getJulianDayNumber + + val hour = calendar.get(Calendar.HOUR_OF_DAY) + val minute = calendar.get(Calendar.MINUTE) + val second = calendar.get(Calendar.SECOND) + val nanos = ts.getNanos + // Hive-0.14 would use hours directly, that might be wrong, since the day starts + // from 12h in Julian. here we just follow what hive does. + val nanosOfDay = nanos + second * NANOS_PER_SECOND + + minute * NANOS_PER_SECOND * SECONDS_PER_MINUTE + + hour * NANOS_PER_SECOND * SECONDS_PER_MINUTE * MINUTES_PER_HOUR + NanoTime(julianDay, nanosOfDay).toBinary + } +} + /** * A `parquet.io.api.GroupConverter` that converts a single-element groups that * match the characteristics of an array (see 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 a54485e719dad..b0db9943a506c 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 @@ -65,8 +65,8 @@ private[sql] case class ParquetRelation( ParquetTypesConverter.readSchemaFromFile( new Path(path.split(",").head), conf, - sqlContext.conf.isParquetBinaryAsString) - + sqlContext.conf.isParquetBinaryAsString, + sqlContext.conf.isParquetINT96AsTimestamp) lazy val attributeMap = AttributeMap(output.map(o => o -> o)) override def newInstance() = ParquetRelation(path, conf, sqlContext).asInstanceOf[this.type] 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 fd63ad8144064..3fb1cc410521e 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 @@ -83,7 +83,8 @@ private[parquet] class RowReadSupport extends ReadSupport[Row] with Logging { // TODO: Why it can be null? if (schema == null) { log.debug("falling back to Parquet read schema") - schema = ParquetTypesConverter.convertToAttributes(parquetSchema, false) + schema = ParquetTypesConverter.convertToAttributes( + parquetSchema, false, true) } log.debug(s"list of attributes that will be read: $schema") new RowRecordMaterializer(parquetSchema, schema) @@ -184,12 +185,12 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { case t @ StructType(_) => writeStruct( t, value.asInstanceOf[CatalystConverter.StructScalaType[_]]) - case _ => writePrimitive(schema.asInstanceOf[PrimitiveType], value) + case _ => writePrimitive(schema.asInstanceOf[NativeType], value) } } } - private[parquet] def writePrimitive(schema: PrimitiveType, value: Any): Unit = { + private[parquet] def writePrimitive(schema: DataType, value: Any): Unit = { if (value != null) { schema match { case StringType => writer.addBinary( @@ -202,6 +203,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { case IntegerType => writer.addInteger(value.asInstanceOf[Int]) case ShortType => writer.addInteger(value.asInstanceOf[Short]) case LongType => writer.addLong(value.asInstanceOf[Long]) + case TimestampType => writeTimestamp(value.asInstanceOf[java.sql.Timestamp]) case ByteType => writer.addInteger(value.asInstanceOf[Byte]) case DoubleType => writer.addDouble(value.asInstanceOf[Double]) case FloatType => writer.addFloat(value.asInstanceOf[Float]) @@ -307,6 +309,10 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { writer.addBinary(Binary.fromByteArray(scratchBytes, 0, numBytes)) } + private[parquet] def writeTimestamp(ts: java.sql.Timestamp): Unit = { + val binaryNanoTime = CatalystTimestampConverter.convertFromTimestamp(ts) + writer.addBinary(binaryNanoTime) + } } // Optimized for non-nested rows @@ -351,6 +357,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 TimestampType => writeTimestamp(record(index).asInstanceOf[java.sql.Timestamp]) case d: DecimalType => if (d.precisionInfo == None || d.precisionInfo.get.precision > 18) { sys.error(s"Unsupported datatype $d, cannot write to consumer") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index d5993656e0225..e4a10aa2ae6c3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.mapreduce.Job import org.apache.spark.sql.test.TestSQLContext import parquet.example.data.{GroupWriter, Group} -import parquet.example.data.simple.SimpleGroup +import parquet.example.data.simple.{NanoTime, SimpleGroup} import parquet.hadoop.{ParquetReader, ParquetFileReader, ParquetWriter} import parquet.hadoop.api.WriteSupport import parquet.hadoop.api.WriteSupport.WriteContext @@ -63,6 +63,7 @@ private[sql] object ParquetTestData { optional int64 mylong; optional float myfloat; optional double mydouble; + optional int96 mytimestamp; }""" // field names for test assertion error messages @@ -72,7 +73,8 @@ private[sql] object ParquetTestData { "mystring:String", "mylong:Long", "myfloat:Float", - "mydouble:Double" + "mydouble:Double", + "mytimestamp:Timestamp" ) val subTestSchema = @@ -98,6 +100,7 @@ private[sql] object ParquetTestData { optional int64 myoptlong; optional float myoptfloat; optional double myoptdouble; + optional int96 mytimestamp; } """ @@ -236,6 +239,7 @@ private[sql] object ParquetTestData { 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() 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 6d8c682ccced8..f1d4ff2387709 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 @@ -54,7 +54,8 @@ private[parquet] object ParquetTypesConverter extends Logging { def toPrimitiveDataType( parquetType: ParquetPrimitiveType, - binaryAsString: Boolean): DataType = { + binaryAsString: Boolean, + int96AsTimestamp: Boolean): DataType = { val originalType = parquetType.getOriginalType val decimalInfo = parquetType.getDecimalMetadata parquetType.getPrimitiveTypeName match { @@ -66,6 +67,7 @@ private[parquet] object ParquetTypesConverter extends Logging { case ParquetPrimitiveTypeName.FLOAT => FloatType case ParquetPrimitiveTypeName.INT32 => IntegerType case ParquetPrimitiveTypeName.INT64 => LongType + case ParquetPrimitiveTypeName.INT96 if int96AsTimestamp => TimestampType case ParquetPrimitiveTypeName.INT96 => // TODO: add BigInteger type? TODO(andre) use DecimalType instead???? sys.error("Potential loss of precision: cannot convert INT96") @@ -103,7 +105,9 @@ private[parquet] object ParquetTypesConverter extends Logging { * @param parquetType The type to convert. * @return The corresponding Catalyst type. */ - def toDataType(parquetType: ParquetType, isBinaryAsString: Boolean): DataType = { + def toDataType(parquetType: ParquetType, + isBinaryAsString: Boolean, + isInt96AsTimestamp: Boolean): DataType = { def correspondsToMap(groupType: ParquetGroupType): Boolean = { if (groupType.getFieldCount != 1 || groupType.getFields.apply(0).isPrimitive) { false @@ -125,7 +129,7 @@ private[parquet] object ParquetTypesConverter extends Logging { } if (parquetType.isPrimitive) { - toPrimitiveDataType(parquetType.asPrimitiveType, isBinaryAsString) + toPrimitiveDataType(parquetType.asPrimitiveType, isBinaryAsString, isInt96AsTimestamp) } else { val groupType = parquetType.asGroupType() parquetType.getOriginalType match { @@ -137,9 +141,12 @@ private[parquet] object ParquetTypesConverter extends Logging { if (field.getName == CatalystConverter.ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME) { val bag = field.asGroupType() assert(bag.getFieldCount == 1) - ArrayType(toDataType(bag.getFields.apply(0), isBinaryAsString), containsNull = true) + ArrayType( + toDataType(bag.getFields.apply(0), isBinaryAsString, isInt96AsTimestamp), + containsNull = true) } else { - ArrayType(toDataType(field, isBinaryAsString), containsNull = false) + ArrayType( + toDataType(field, isBinaryAsString, isInt96AsTimestamp), containsNull = false) } } case ParquetOriginalType.MAP => { @@ -152,8 +159,10 @@ private[parquet] object ParquetTypesConverter extends Logging { "Parquet Map type malformatted: nested group should have 2 (key, value) fields!") assert(keyValueGroup.getFields.apply(0).getRepetition == Repetition.REQUIRED) - val keyType = toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString) - val valueType = toDataType(keyValueGroup.getFields.apply(1), isBinaryAsString) + val keyType = + toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString, isInt96AsTimestamp) + val valueType = + toDataType(keyValueGroup.getFields.apply(1), isBinaryAsString, isInt96AsTimestamp) MapType(keyType, valueType, keyValueGroup.getFields.apply(1).getRepetition != Repetition.REQUIRED) } @@ -163,8 +172,10 @@ private[parquet] object ParquetTypesConverter extends Logging { val keyValueGroup = groupType.getFields.apply(0).asGroupType() assert(keyValueGroup.getFields.apply(0).getRepetition == Repetition.REQUIRED) - val keyType = toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString) - val valueType = toDataType(keyValueGroup.getFields.apply(1), isBinaryAsString) + val keyType = + toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString, isInt96AsTimestamp) + val valueType = + toDataType(keyValueGroup.getFields.apply(1), isBinaryAsString, isInt96AsTimestamp) MapType(keyType, valueType, keyValueGroup.getFields.apply(1).getRepetition != Repetition.REQUIRED) } else if (correspondsToArray(groupType)) { // ArrayType @@ -172,16 +183,19 @@ private[parquet] object ParquetTypesConverter extends Logging { if (field.getName == CatalystConverter.ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME) { val bag = field.asGroupType() assert(bag.getFieldCount == 1) - ArrayType(toDataType(bag.getFields.apply(0), isBinaryAsString), containsNull = true) + ArrayType( + toDataType(bag.getFields.apply(0), isBinaryAsString, isInt96AsTimestamp), + containsNull = true) } else { - ArrayType(toDataType(field, isBinaryAsString), containsNull = false) + ArrayType( + toDataType(field, isBinaryAsString, isInt96AsTimestamp), containsNull = false) } } else { // everything else: StructType val fields = groupType .getFields .map(ptype => new StructField( ptype.getName, - toDataType(ptype, isBinaryAsString), + toDataType(ptype, isBinaryAsString, isInt96AsTimestamp), ptype.getRepetition != Repetition.REQUIRED)) StructType(fields) } @@ -210,6 +224,7 @@ private[parquet] object ParquetTypesConverter extends Logging { case ShortType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.INT32)) case ByteType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.INT32)) case LongType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.INT64)) + case TimestampType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.INT96)) case DecimalType.Fixed(precision, scale) if precision <= 18 => // TODO: for now, our writer only supports decimals that fit in a Long Some(ParquetTypeInfo(ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, @@ -345,7 +360,9 @@ private[parquet] object ParquetTypesConverter extends Logging { } } - def convertToAttributes(parquetSchema: ParquetType, isBinaryAsString: Boolean): Seq[Attribute] = { + def convertToAttributes(parquetSchema: ParquetType, + isBinaryAsString: Boolean, + isInt96AsTimestamp: Boolean): Seq[Attribute] = { parquetSchema .asGroupType() .getFields @@ -353,7 +370,7 @@ private[parquet] object ParquetTypesConverter extends Logging { field => new AttributeReference( field.getName, - toDataType(field, isBinaryAsString), + toDataType(field, isBinaryAsString, isInt96AsTimestamp), field.getRepetition != Repetition.REQUIRED)()) } @@ -476,7 +493,8 @@ private[parquet] object ParquetTypesConverter extends Logging { def readSchemaFromFile( origPath: Path, conf: Option[Configuration], - isBinaryAsString: Boolean): Seq[Attribute] = { + isBinaryAsString: Boolean, + isInt96AsTimestamp: Boolean): Seq[Attribute] = { val keyValueMetadata: java.util.Map[String, String] = readMetaData(origPath, conf) .getFileMetaData @@ -485,7 +503,9 @@ private[parquet] object ParquetTypesConverter extends Logging { convertFromString(keyValueMetadata.get(RowReadSupport.SPARK_METADATA_KEY)) } else { val attributes = convertToAttributes( - readMetaData(origPath, conf).getFileMetaData.getSchema, isBinaryAsString) + readMetaData(origPath, conf).getFileMetaData.getSchema, + isBinaryAsString, + isInt96AsTimestamp) log.info(s"Falling back to schema conversion from Parquet types; result: $attributes") attributes } 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 1e794cad73936..179c0d6b22239 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 @@ -136,7 +136,8 @@ case class ParquetRelation2(path: String)(@transient val sqlContext: SQLContext) ParquetTypesConverter.readSchemaFromFile( partitions.head.files.head.getPath, Some(sparkContext.hadoopConfiguration), - sqlContext.conf.isParquetBinaryAsString)) + sqlContext.conf.isParquetBinaryAsString, + sqlContext.conf.isParquetINT96AsTimestamp)) val dataIncludesKey = partitionKeys.headOption.map(dataSchema.fieldNames.contains(_)).getOrElse(true) 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 new file mode 100644 index 0000000000000..887161684429f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/timestamp/NanoTime.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.sql.parquet.timestamp + +import java.nio.{ByteBuffer, ByteOrder} + +import parquet.Preconditions +import parquet.io.api.{Binary, RecordConsumer} + +private[parquet] class NanoTime extends Serializable { + private var julianDay = 0 + private var timeOfDayNanos = 0L + + def set(julianDay: Int, timeOfDayNanos: Long) = { + this.julianDay = julianDay + this.timeOfDayNanos = timeOfDayNanos + this + } + + def getJulianDay: Int = julianDay + + def getTimeOfDayNanos: Long = timeOfDayNanos + + def toBinary: Binary = { + val buf = ByteBuffer.allocate(12) + buf.order(ByteOrder.LITTLE_ENDIAN) + buf.putLong(timeOfDayNanos) + buf.putInt(julianDay) + buf.flip() + Binary.fromByteBuffer(buf) + } + + def writeValue(recordConsumer: RecordConsumer) { + recordConsumer.addBinary(toBinary) + } + + override def toString = + "NanoTime{julianDay=" + julianDay + ", timeOfDayNanos=" + timeOfDayNanos + "}" +} + +object NanoTime { + def fromBinary(bytes: Binary): NanoTime = { + Preconditions.checkArgument(bytes.length() == 12, "Must be 12 bytes") + val buf = bytes.toByteBuffer + buf.order(ByteOrder.LITTLE_ENDIAN) + val timeOfDayNanos = buf.getLong + val julianDay = buf.getInt + new NanoTime().set(julianDay, timeOfDayNanos) + } + + def apply(julianDay: Int, timeOfDayNanos: Long): NanoTime = { + new NanoTime().set(julianDay, timeOfDayNanos) + } +} From ca7a6cdff004eb4605fd223e127b4a46a0a214e7 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 3 Feb 2015 12:12:26 -0800 Subject: [PATCH 046/578] [SPARK-5550] [SQL] Support the case insensitive for UDF SQL in HiveContext, should be case insensitive, however, the following query will fail. ```scala udf.register("random0", () => { Math.random()}) assert(sql("SELECT RANDOM0() FROM src LIMIT 1").head().getDouble(0) >= 0.0) ``` Author: Cheng Hao Closes #4326 from chenghao-intel/udf_case_sensitive and squashes the following commits: 485cf66 [Cheng Hao] Support the case insensitive for UDF --- .../catalyst/analysis/FunctionRegistry.scala | 36 ++++++++++++++++--- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 4 ++- .../org/apache/spark/sql/hive/UDFSuite.scala | 36 +++++++++++++++++++ 4 files changed, 72 insertions(+), 6 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala 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 760c49fbca4a5..9f334f6d42ad1 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 @@ -27,23 +27,25 @@ trait FunctionRegistry { def registerFunction(name: String, builder: FunctionBuilder): Unit def lookupFunction(name: String, children: Seq[Expression]): Expression + + def caseSensitive: Boolean } trait OverrideFunctionRegistry extends FunctionRegistry { - val functionBuilders = new mutable.HashMap[String, FunctionBuilder]() + val functionBuilders = StringKeyHashMap[FunctionBuilder](caseSensitive) def registerFunction(name: String, builder: FunctionBuilder) = { functionBuilders.put(name, builder) } abstract override def lookupFunction(name: String, children: Seq[Expression]): Expression = { - functionBuilders.get(name).map(_(children)).getOrElse(super.lookupFunction(name,children)) + functionBuilders.get(name).map(_(children)).getOrElse(super.lookupFunction(name, children)) } } -class SimpleFunctionRegistry extends FunctionRegistry { - val functionBuilders = new mutable.HashMap[String, FunctionBuilder]() +class SimpleFunctionRegistry(val caseSensitive: Boolean) extends FunctionRegistry { + val functionBuilders = StringKeyHashMap[FunctionBuilder](caseSensitive) def registerFunction(name: String, builder: FunctionBuilder) = { functionBuilders.put(name, builder) @@ -64,4 +66,30 @@ object EmptyFunctionRegistry extends FunctionRegistry { def lookupFunction(name: String, children: Seq[Expression]): Expression = { throw new UnsupportedOperationException } + + def caseSensitive: Boolean = ??? +} + +/** + * Build a map with String type of key, and it also supports either key case + * sensitive or insensitive. + * TODO move this into util folder? + */ +object StringKeyHashMap { + def apply[T](caseSensitive: Boolean) = caseSensitive match { + case false => new StringKeyHashMap[T](_.toLowerCase) + case true => new StringKeyHashMap[T](identity) + } +} + +class StringKeyHashMap[T](normalizer: (String) => String) { + private val base = new collection.mutable.HashMap[String, T]() + + def apply(key: String): T = base(normalizer(key)) + + def get(key: String): Option[T] = base.get(normalizer(key)) + def put(key: String, value: T): Option[T] = base.put(normalizer(key), value) + def remove(key: String): Option[T] = base.remove(normalizer(key)) + def iterator: Iterator[(String, T)] = base.toIterator } + 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 a741d0031d155..2697e780c05c6 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 @@ -87,7 +87,7 @@ class SQLContext(@transient val sparkContext: SparkContext) protected[sql] lazy val catalog: Catalog = new SimpleCatalog(true) @transient - protected[sql] lazy val functionRegistry: FunctionRegistry = new SimpleFunctionRegistry + protected[sql] lazy val functionRegistry: FunctionRegistry = new SimpleFunctionRegistry(true) @transient protected[sql] lazy val analyzer: Analyzer = 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 f6d9027f90a99..50f266a4bc29e 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 @@ -311,7 +311,9 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { // Note that HiveUDFs will be overridden by functions registered in this context. @transient override protected[sql] lazy val functionRegistry = - new HiveFunctionRegistry with OverrideFunctionRegistry + new HiveFunctionRegistry with OverrideFunctionRegistry { + def caseSensitive = false + } /* An analyzer that uses the Hive metastore. */ @transient diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala new file mode 100644 index 0000000000000..85b6bc93d7122 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +/* Implicits */ + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.hive.test.TestHive._ + +case class FunctionResult(f1: String, f2: String) + +class UDFSuite extends QueryTest { + test("UDF case insensitive") { + udf.register("random0", () => { Math.random()}) + udf.register("RANDOM1", () => { Math.random()}) + udf.register("strlenScala", (_: String).length + (_:Int)) + assert(sql("SELECT RANDOM0() FROM src LIMIT 1").head().getDouble(0) >= 0.0) + assert(sql("SELECT RANDOm1() FROM src LIMIT 1").head().getDouble(0) >= 0.0) + assert(sql("SELECT strlenscala('test', 1) FROM src LIMIT 1").head().getInt(0) === 5) + } +} From 5adbb39482631998dbfe4a1da88f6e75b30fb5ac Mon Sep 17 00:00:00 2001 From: wangfei Date: Tue, 3 Feb 2015 12:16:31 -0800 Subject: [PATCH 047/578] [SPARK-5383][SQL] Support alias for udtfs Add support for alias of udtfs, such as ``` select stack(2, key, value, key, value) as (a, b) from src limit 5; select a, b from (select stack(2, key, value, key, value) as (a, b) from src) t limit 5 ``` Author: wangfei Author: scwf Author: Fei Wang Closes #4186 from scwf/multi-alias-names and squashes the following commits: c35e922 [wangfei] fix conflicts adc8311 [wangfei] minor format fix 2783aed [wangfei] convert it to a Generate instead of leaving it inside of a Project clause a87668a [wangfei] minor improvement b25d9b3 [wangfei] resolve conflicts d38f041 [wangfei] style fix 8cfcebf [wangfei] minor improvement 12a239e [wangfei] fix test case 050177f [wangfei] added extendedCheckRules 3d69329 [wangfei] added CheckMultiAlias to analyzer 324150d [wangfei] added multi alias node 74f5a81 [Fei Wang] imports order fix 5bc3f59 [scwf] style fix 3daec28 [scwf] support alias for udfs with multi output columns --- .../sql/catalyst/analysis/Analyzer.scala | 5 ++- .../sql/catalyst/analysis/unresolved.scala | 38 +++++++++++++++++++ .../expressions/namedExpressions.scala | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 1 + .../org/apache/spark/sql/hive/HiveQl.scala | 16 ++++++-- .../org/apache/spark/sql/hive/hiveUdfs.scala | 19 +++++++++- .../sql/hive/execution/HiveQuerySuite.scala | 12 ++++++ 7 files changed, 85 insertions(+), 8 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 cefd70acf3931..ae7f7b9feb5fd 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 @@ -69,8 +69,9 @@ class Analyzer(catalog: Catalog, typeCoercionRules ++ extendedRules : _*), Batch("Check Analysis", Once, - CheckResolution, - CheckAggregation), + CheckResolution :: + CheckAggregation :: + Nil: _*), Batch("AnalysisOperators", fixedPoint, EliminateAnalysisOperators) ) 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 66060289189ef..f35921e2a772c 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 @@ -128,6 +128,44 @@ case class UnresolvedStar(table: Option[String]) extends Star { override def toString = table.map(_ + ".").getOrElse("") + "*" } +/** + * Used to assign new names to Generator's output, such as hive udtf. + * For example the SQL expression "stack(2, key, value, key, value) as (a, b)" could be represented + * as follows: + * MultiAlias(stack_function, Seq(a, b)) + + * @param child the computation being performed + * @param names the names to be associated with each output of computing [[child]]. + */ +case class MultiAlias(child: Expression, names: Seq[String]) + extends Attribute with trees.UnaryNode[Expression] { + + 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 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 eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + + override def toString: String = s"$child AS $names" + +} /** * Represents all the resolved input attributes to a given relational operator. This is used 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 f388cd5972bac..e6ab1fd8d7939 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 @@ -75,7 +75,7 @@ abstract class Attribute extends NamedExpression { /** * Used to assign a new name to a computation. * For example the SQL expression "1 + 1 AS a" could be represented as follows: - * Alias(Add(Literal(1), Literal(1), "a")() + * Alias(Add(Literal(1), Literal(1)), "a")() * * Note that exprId and qualifiers are in a separate parameter list because * we only pattern match on child and name. 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 50f266a4bc29e..922e61f0be12f 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 @@ -323,6 +323,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { catalog.CreateTables :: catalog.PreInsertionCasts :: ExtractPythonUdfs :: + ResolveUdtfsAlias :: 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 ab305e1f82a55..74ca0d4ed5bb1 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 import org.apache.hadoop.hive.ql.Context @@ -968,14 +969,21 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C } protected def selExprNodeToExpr(node: Node): Option[Expression] = node match { - case Token("TOK_SELEXPR", - e :: Nil) => + case Token("TOK_SELEXPR", e :: Nil) => Some(nodeToExpr(e)) - case Token("TOK_SELEXPR", - e :: Token(alias, Nil) :: Nil) => + case Token("TOK_SELEXPR", e :: Token(alias, Nil) :: Nil) => Some(Alias(nodeToExpr(e), cleanIdentifier(alias))()) + case Token("TOK_SELEXPR", e :: aliasChildren) => + var aliasNames = ArrayBuffer[String]() + aliasChildren.foreach { _ match { + case Token(name, Nil) => aliasNames += cleanIdentifier(name) + case _ => + } + } + Some(MultiAlias(nodeToExpr(e), aliasNames)) + /* Hints are ignored */ case Token("TOK_HINTLIST", _) => None 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 76d2140372197..34c21c11761ae 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 @@ -33,8 +33,11 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ import org.apache.spark.Logging import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.{Generate, Project, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.types._ -import org.apache.spark.util.Utils.getContextOrSparkClassLoader +import org.apache.spark.sql.catalyst.analysis.MultiAlias +import org.apache.spark.sql.catalyst.errors.TreeNodeException /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -321,6 +324,20 @@ private[hive] case class HiveGenericUdtf( override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" } +/** + * Resolve Udtfs Alias. + */ +private[spark] object ResolveUdtfsAlias extends Rule[LogicalPlan] { + def apply(plan: 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") + + case Project(Seq(MultiAlias(udtf @ HiveGenericUdtf(_, _, _), names)), child) => + Generate(udtf.copy(aliasNames = names), join = false, outer = false, None, child) + } +} + private[hive] case class HiveUdafFunction( funcWrapper: HiveFunctionWrapper, exprs: Seq[Expression], 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 4c53b10ba96e9..8e84d279fe119 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 @@ -583,6 +583,18 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { assert(sql("select key from src having key > 490").collect().size < 100) } + test("SPARK-5383 alias for udfs with multi output columns") { + assert( + sql("select stack(2, key, value, key, value) as (a, b) from src limit 5") + .collect() + .size == 5) + + assert( + sql("select a, b from (select stack(2, key, value, key, value) as (a, b) from src) t limit 5") + .collect() + .size == 5) + } + test("SPARK-5367: resolve star expression in udf") { assert(sql("select concat(*) from src limit 5").collect().size == 5) assert(sql("select array(*) from src limit 5").collect().size == 5) From db821ed2ededf6ce79b838c77a9c10bed2ce555a Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Tue, 3 Feb 2015 12:21:45 -0800 Subject: [PATCH 048/578] [SPARK-4508] [SQL] build native date type to conform behavior to Hive The previous #3732 is reverted due to some test failure. Have fixed that. Author: Daoyuan Wang Closes #4325 from adrian-wang/datenative and squashes the following commits: 096e20d [Daoyuan Wang] fix for mixed timezone 0ed0fdc [Daoyuan Wang] fix test data a2fdd4e [Daoyuan Wang] getDate c37832b [Daoyuan Wang] row to catalyst f0005b1 [Daoyuan Wang] add date in sql parser and java type conversion 024c9a6 [Daoyuan Wang] clean some import order d6715fc [Daoyuan Wang] refactoring Date as Primitive Int internally 374abd5 [Daoyuan Wang] spark native date type support --- .../main/scala/org/apache/spark/sql/Row.scala | 2 +- .../spark/sql/catalyst/ScalaReflection.scala | 9 +-- .../apache/spark/sql/catalyst/SqlParser.scala | 2 + .../spark/sql/catalyst/expressions/Cast.scala | 53 +++++++--------- .../expressions/codegen/CodeGenerator.scala | 3 + .../sql/catalyst/expressions/literals.scala | 2 +- .../apache/spark/sql/types/DateUtils.scala | 60 +++++++++++++++++++ .../apache/spark/sql/types/dataTypes.scala | 12 ++-- .../ExpressionEvaluationSuite.scala | 28 ++++----- .../spark/sql/types/DataTypeSuite.scala | 2 +- .../spark/sql/columnar/ColumnStats.scala | 19 +----- .../spark/sql/columnar/ColumnType.scala | 13 ++-- .../spark/sql/execution/pythonUdfs.scala | 4 +- .../org/apache/spark/sql/json/JsonRDD.scala | 6 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 33 ++++++---- .../sql/ScalaReflectionRelationSuite.scala | 3 +- .../spark/sql/columnar/ColumnStatsSuite.scala | 2 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 6 +- .../sql/columnar/ColumnarTestUtils.scala | 6 +- .../org/apache/spark/sql/json/JsonSuite.scala | 7 ++- .../execution/HiveCompatibilitySuite.scala | 1 + .../apache/spark/sql/hive/HiveContext.scala | 4 +- .../spark/sql/hive/HiveInspectors.scala | 11 +++- .../apache/spark/sql/hive/TableReader.scala | 3 +- ...te cast-0-a7cd69b80c77a771a2c955db666be53d | 1 + ... test 1-0-bde89be08a12361073ff658fef768b7e | 1 + ... test 2-0-dc1b267f1d79d49e6675afe4fd2a34a5 | 1 + .../date_1-0-50131c0ba7b7a6b65c789a5a8497bada | 1 + ...date_1-1-23edf29bf7376c70d5ecf12720f4b1eb} | 0 ...ate_1-10-df16364a220ff96a6ea1cd478cbc1d0b} | 0 ...ate_1-11-d964bec7e5632091ab5cb6f6786dbbf9} | 0 ...ate_1-12-480c5f024a28232b7857be327c992509} | 0 ...ate_1-13-4c0ed7fcb75770d8790575b586bf14f4} | 0 ...date_1-14-44fc74c1993062c0a9522199ff27fea} | 0 ...ate_1-15-4855a66124b16d1d0d003235995ac06b} | 0 ...ate_1-16-8bc190dba0f641840b5e1e198a14c55b} | 0 ...ate_1-17-23edf29bf7376c70d5ecf12720f4b1eb} | 0 ...date_1-2-4ebe3571c13a8b0c03096fbd972b7f1b} | 0 ... date_1-3-26b5c291400dfde455b3c1b878b71d0} | 0 ...date_1-4-df16364a220ff96a6ea1cd478cbc1d0b} | 0 ...date_1-5-d964bec7e5632091ab5cb6f6786dbbf9} | 0 ...date_1-6-559d01fb0b42c42f0c4927fa0f9deac4} | 0 ...date_1-7-df16364a220ff96a6ea1cd478cbc1d0b} | 0 ...date_1-8-d964bec7e5632091ab5cb6f6786dbbf9} | 0 ...date_1-9-8306558e0eabe936ac33dabaaa17fea4} | 0 .../spark/sql/hive/HiveInspectorSuite.scala | 4 +- .../sql/hive/execution/HiveQuerySuite.scala | 26 +++++++- .../org/apache/spark/sql/hive/Shim12.scala | 2 +- .../org/apache/spark/sql/hive/Shim13.scala | 2 +- 49 files changed, 204 insertions(+), 125 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala create mode 100644 sql/hive/src/test/resources/golden/Date cast-0-a7cd69b80c77a771a2c955db666be53d create mode 100644 sql/hive/src/test/resources/golden/Date comparison test 1-0-bde89be08a12361073ff658fef768b7e create mode 100644 sql/hive/src/test/resources/golden/Date comparison test 2-0-dc1b267f1d79d49e6675afe4fd2a34a5 create mode 100644 sql/hive/src/test/resources/golden/date_1-0-50131c0ba7b7a6b65c789a5a8497bada rename sql/hive/src/test/resources/golden/{date_1-0-23edf29bf7376c70d5ecf12720f4b1eb => date_1-1-23edf29bf7376c70d5ecf12720f4b1eb} (100%) rename sql/hive/src/test/resources/golden/{date_1-3-df16364a220ff96a6ea1cd478cbc1d0b => date_1-10-df16364a220ff96a6ea1cd478cbc1d0b} (100%) rename sql/hive/src/test/resources/golden/{date_1-10-d964bec7e5632091ab5cb6f6786dbbf9 => date_1-11-d964bec7e5632091ab5cb6f6786dbbf9} (100%) rename sql/hive/src/test/resources/golden/{date_1-11-480c5f024a28232b7857be327c992509 => date_1-12-480c5f024a28232b7857be327c992509} (100%) rename sql/hive/src/test/resources/golden/{date_1-12-4c0ed7fcb75770d8790575b586bf14f4 => date_1-13-4c0ed7fcb75770d8790575b586bf14f4} (100%) rename sql/hive/src/test/resources/golden/{date_1-13-44fc74c1993062c0a9522199ff27fea => date_1-14-44fc74c1993062c0a9522199ff27fea} (100%) rename sql/hive/src/test/resources/golden/{date_1-14-4855a66124b16d1d0d003235995ac06b => date_1-15-4855a66124b16d1d0d003235995ac06b} (100%) rename sql/hive/src/test/resources/golden/{date_1-15-8bc190dba0f641840b5e1e198a14c55b => date_1-16-8bc190dba0f641840b5e1e198a14c55b} (100%) rename sql/hive/src/test/resources/golden/{date_1-1-4ebe3571c13a8b0c03096fbd972b7f1b => date_1-17-23edf29bf7376c70d5ecf12720f4b1eb} (100%) rename sql/hive/src/test/resources/golden/{date_1-16-23edf29bf7376c70d5ecf12720f4b1eb => date_1-2-4ebe3571c13a8b0c03096fbd972b7f1b} (100%) rename sql/hive/src/test/resources/golden/{date_1-2-abdce0c0d14d3fc7441b7c134b02f99a => date_1-3-26b5c291400dfde455b3c1b878b71d0} (100%) rename sql/hive/src/test/resources/golden/{date_1-6-df16364a220ff96a6ea1cd478cbc1d0b => date_1-4-df16364a220ff96a6ea1cd478cbc1d0b} (100%) rename sql/hive/src/test/resources/golden/{date_1-4-d964bec7e5632091ab5cb6f6786dbbf9 => date_1-5-d964bec7e5632091ab5cb6f6786dbbf9} (100%) rename sql/hive/src/test/resources/golden/{date_1-5-5e70fc74158fbfca38134174360de12d => date_1-6-559d01fb0b42c42f0c4927fa0f9deac4} (100%) rename sql/hive/src/test/resources/golden/{date_1-9-df16364a220ff96a6ea1cd478cbc1d0b => date_1-7-df16364a220ff96a6ea1cd478cbc1d0b} (100%) rename sql/hive/src/test/resources/golden/{date_1-7-d964bec7e5632091ab5cb6f6786dbbf9 => date_1-8-d964bec7e5632091ab5cb6f6786dbbf9} (100%) rename sql/hive/src/test/resources/golden/{date_1-8-1d5c58095cd52ea539d869f2ab1ab67d => date_1-9-8306558e0eabe936ac33dabaaa17fea4} (100%) 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 41bb4f012f2e1..3a70d25534968 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 object Row { /** 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 e0db587efb08d..8e79e532ca564 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 @@ -17,14 +17,13 @@ package org.apache.spark.sql.catalyst -import java.sql.{Date, Timestamp} +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.plans.logical.LocalRelation import org.apache.spark.sql.types._ - /** * A default version of ScalaReflection that uses the runtime universe. */ @@ -72,6 +71,7 @@ trait ScalaReflection { }.toArray) case (d: BigDecimal, _) => Decimal(d) case (d: java.math.BigDecimal, _) => Decimal(d) + case (d: java.sql.Date, _) => DateUtils.fromJavaDate(d) case (other, _) => other } @@ -85,6 +85,7 @@ trait ScalaReflection { } case (r: Row, s: StructType) => convertRowToScala(r, s) case (d: Decimal, _: DecimalType) => d.toJavaBigDecimal + case (i: Int, DateType) => DateUtils.toJavaDate(i) case (other, _) => other } @@ -159,7 +160,7 @@ trait ScalaReflection { 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[Date] => Schema(DateType, nullable = true) + case t if t <:< typeOf[java.sql.Date] => Schema(DateType, nullable = true) case t if t <:< typeOf[BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) case t if t <:< typeOf[java.math.BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) case t if t <:< typeOf[Decimal] => Schema(DecimalType.Unlimited, nullable = true) @@ -191,7 +192,7 @@ trait ScalaReflection { case obj: LongType.JvmType => LongType case obj: FloatType.JvmType => FloatType case obj: DoubleType.JvmType => DoubleType - case obj: DateType.JvmType => DateType + case obj: java.sql.Date => DateType case obj: java.math.BigDecimal => DecimalType.Unlimited case obj: Decimal => DecimalType.Unlimited case obj: TimestampType.JvmType => TimestampType 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 25e639d390da0..5c006e9d4c6f5 100755 --- 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 @@ -52,6 +52,7 @@ 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") @@ -383,6 +384,7 @@ class SqlParser extends AbstractSparkSQLParser { | DOUBLE ^^^ DoubleType | fixedDecimalType | DECIMAL ^^^ DecimalType.Unlimited + | DATE ^^^ DateType ) protected lazy val fixedDecimalType: Parser[DataType] = 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 ece5ee73618cb..b1bc858478ee1 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 @@ -113,7 +113,7 @@ 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[Date](_, dateToString) + case DateType => buildCast[Int](_, d => DateUtils.toString(d)) case TimestampType => buildCast[Timestamp](_, timestampToString) case _ => buildCast[Any](_, _.toString) } @@ -131,7 +131,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w buildCast[Timestamp](_, t => t.getTime() != 0 || t.getNanos() != 0) case DateType => // Hive would return null when cast from date to boolean - buildCast[Date](_, d => null) + buildCast[Int](_, d => null) case LongType => buildCast[Long](_, _ != 0) case IntegerType => @@ -171,7 +171,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case ByteType => buildCast[Byte](_, b => new Timestamp(b)) case DateType => - buildCast[Date](_, d => new Timestamp(d.getTime)) + buildCast[Int](_, d => new Timestamp(DateUtils.toJavaDate(d).getTime)) // TimestampWritable.decimalToTimestamp case DecimalType() => buildCast[Decimal](_, d => decimalToTimestamp(d)) @@ -224,37 +224,24 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w } } - // Converts Timestamp to string according to Hive TimestampWritable convention - private[this] def timestampToDateString(ts: Timestamp): String = { - Cast.threadLocalDateFormat.get.format(ts) - } - // DateConverter private[this] def castToDate(from: DataType): Any => Any = from match { case StringType => buildCast[String](_, s => - try Date.valueOf(s) catch { case _: java.lang.IllegalArgumentException => null }) + try DateUtils.fromJavaDate(Date.valueOf(s)) + catch { case _: java.lang.IllegalArgumentException => null } + ) case TimestampType => // throw valid precision more than seconds, according to Hive. // Timestamp.nanos is in 0 to 999,999,999, no more than a second. - buildCast[Timestamp](_, t => new Date(Math.floor(t.getTime / 1000.0).toLong * 1000)) + buildCast[Timestamp](_, t => DateUtils.millisToDays(t.getTime)) // Hive throws this exception as a Semantic Exception - // It is never possible to compare result when hive return with exception, so we can return null + // It is never possible to compare result when hive return with exception, + // so we can return null // NULL is more reasonable here, since the query itself obeys the grammar. case _ => _ => null } - // Date cannot be cast to long, according to hive - private[this] def dateToLong(d: Date) = null - - // Date cannot be cast to double, according to hive - private[this] def dateToDouble(d: Date) = null - - // Converts Date to string according to Hive DateWritable convention - private[this] def dateToString(d: Date): String = { - Cast.threadLocalDateFormat.get.format(d) - } - // LongConverter private[this] def castToLong(from: DataType): Any => Any = from match { case StringType => @@ -264,7 +251,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => if (b) 1L else 0L) case DateType => - buildCast[Date](_, d => dateToLong(d)) + buildCast[Int](_, d => null) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t)) case x: NumericType => @@ -280,7 +267,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => if (b) 1 else 0) case DateType => - buildCast[Date](_, d => dateToLong(d)) + buildCast[Int](_, d => null) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toInt) case x: NumericType => @@ -296,7 +283,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => if (b) 1.toShort else 0.toShort) case DateType => - buildCast[Date](_, d => dateToLong(d)) + buildCast[Int](_, d => null) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toShort) case x: NumericType => @@ -312,7 +299,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => if (b) 1.toByte else 0.toByte) case DateType => - buildCast[Date](_, d => dateToLong(d)) + buildCast[Int](_, d => null) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toByte) case x: NumericType => @@ -342,7 +329,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => changePrecision(if (b) Decimal(1) else Decimal(0), target)) case DateType => - buildCast[Date](_, d => null) // date can't cast to decimal in Hive + buildCast[Int](_, d => null) // date can't cast to decimal in Hive case TimestampType => // Note that we lose precision here. buildCast[Timestamp](_, t => changePrecision(Decimal(timestampToDouble(t)), target)) @@ -367,7 +354,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => if (b) 1d else 0d) case DateType => - buildCast[Date](_, d => dateToDouble(d)) + buildCast[Int](_, d => null) case TimestampType => buildCast[Timestamp](_, t => timestampToDouble(t)) case x: NumericType => @@ -383,7 +370,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => if (b) 1f else 0f) case DateType => - buildCast[Date](_, d => dateToDouble(d)) + buildCast[Int](_, d => null) case TimestampType => buildCast[Timestamp](_, t => timestampToDouble(t).toFloat) case x: NumericType => @@ -442,16 +429,16 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w object Cast { // `SimpleDateFormat` is not thread-safe. - private[sql] val threadLocalDateFormat = new ThreadLocal[DateFormat] { + private[sql] val threadLocalTimestampFormat = new ThreadLocal[DateFormat] { override def initialValue() = { - new SimpleDateFormat("yyyy-MM-dd") + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") } } // `SimpleDateFormat` is not thread-safe. - private[sql] val threadLocalTimestampFormat = new ThreadLocal[DateFormat] { + private[sql] val threadLocalDateFormat = new ThreadLocal[DateFormat] { override def initialValue() = { - new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + new SimpleDateFormat("yyyy-MM-dd") } } } 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 4cae5c4718683..1f80d84b744a1 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 @@ -246,6 +246,9 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin new String(${eval.primitiveTerm}.asInstanceOf[Array[Byte]]) """.children + case Cast(child @ DateType(), StringType) => + child.castOrNull(c => q"org.apache.spark.sql.types.DateUtils.toString($c)", StringType) + case Cast(child @ NumericType(), IntegerType) => child.castOrNull(c => q"$c.toInt", IntegerType) 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 5b389aad7a85d..97bb96f48e2c7 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 @@ -35,7 +35,7 @@ object Literal { case d: java.math.BigDecimal => Literal(Decimal(d), DecimalType.Unlimited) case d: Decimal => Literal(d, DecimalType.Unlimited) case t: Timestamp => Literal(t, TimestampType) - case d: Date => Literal(d, DateType) + case d: Date => Literal(DateUtils.fromJavaDate(d), DateType) case a: Array[Byte] => Literal(a, BinaryType) case null => Literal(null, NullType) } 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 new file mode 100644 index 0000000000000..8a1a3b81b3d2c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.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.sql.types + +import java.sql.Date +import java.util.{Calendar, TimeZone} + +import org.apache.spark.sql.catalyst.expressions.Cast + +/** + * helper function to convert between Int value of days since 1970-01-01 and java.sql.Date + */ +object DateUtils { + private val MILLIS_PER_DAY = 86400000 + + // Java TimeZone has no mention of thread safety. Use thread local instance to be safe. + private val LOCAL_TIMEZONE = new ThreadLocal[TimeZone] { + override protected def initialValue: TimeZone = { + Calendar.getInstance.getTimeZone + } + } + + private def javaDateToDays(d: Date): Int = { + millisToDays(d.getTime) + } + + def millisToDays(millisLocal: Long): Int = { + ((millisLocal + LOCAL_TIMEZONE.get().getOffset(millisLocal)) / MILLIS_PER_DAY).toInt + } + + private def toMillisSinceEpoch(days: Int): Long = { + val millisUtc = days.toLong * MILLIS_PER_DAY + millisUtc - LOCAL_TIMEZONE.get().getOffset(millisUtc) + } + + def fromJavaDate(date: java.sql.Date): Int = { + javaDateToDays(date) + } + + def toJavaDate(daysSinceEpoch: Int): java.sql.Date = { + new java.sql.Date(toMillisSinceEpoch(daysSinceEpoch)) + } + + def toString(days: Int): String = Cast.threadLocalDateFormat.get.format(toJavaDate(days)) +} 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 defdcb2b706f5..4825d1ff81402 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.types -import java.sql.{Date, Timestamp} +import java.sql.Timestamp import scala.math.Numeric.{FloatAsIfIntegral, DoubleAsIfIntegral} import scala.reflect.ClassTag @@ -387,18 +387,16 @@ case object TimestampType extends NativeType { */ @DeveloperApi case object DateType extends NativeType { - private[sql] type JvmType = Date + private[sql] type JvmType = Int @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - private[sql] val ordering = new Ordering[JvmType] { - def compare(x: Date, y: Date) = x.compareTo(y) - } + private[sql] val ordering = implicitly[Ordering[JvmType]] /** - * The default size of a value of the DateType is 8 bytes. + * The default size of a value of the DateType is 4 bytes. */ - override def defaultSize: Int = 8 + override def defaultSize: Int = 4 } 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 37e64adeea853..25d1c105a00a6 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 @@ -303,6 +303,7 @@ class ExpressionEvaluationSuite extends FunSuite { val sd = "1970-01-01" val d = Date.valueOf(sd) + val zts = sd + " 00:00:00" val sts = sd + " 00:00:02" val nts = sts + ".1" val ts = Timestamp.valueOf(nts) @@ -319,14 +320,14 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble) checkEvaluation(Cast(Literal(sd) cast DateType, StringType), sd) - checkEvaluation(Cast(Literal(d) cast StringType, DateType), d) + checkEvaluation(Cast(Literal(d) cast StringType, DateType), 0) checkEvaluation(Cast(Literal(nts) cast TimestampType, StringType), nts) checkEvaluation(Cast(Literal(ts) cast StringType, TimestampType), ts) // all convert to string type to check checkEvaluation( Cast(Cast(Literal(nts) cast TimestampType, DateType), StringType), sd) checkEvaluation( - Cast(Cast(Literal(ts) cast DateType, TimestampType), StringType), sts) + Cast(Cast(Literal(ts) cast DateType, TimestampType), StringType), zts) checkEvaluation(Cast("abdef" cast BinaryType, StringType), "abdef") @@ -377,8 +378,8 @@ class ExpressionEvaluationSuite extends FunSuite { } test("date") { - val d1 = Date.valueOf("1970-01-01") - val d2 = Date.valueOf("1970-01-02") + val d1 = DateUtils.fromJavaDate(Date.valueOf("1970-01-01")) + val d2 = DateUtils.fromJavaDate(Date.valueOf("1970-01-02")) checkEvaluation(Literal(d1) < Literal(d2), true) } @@ -459,22 +460,21 @@ class ExpressionEvaluationSuite extends FunSuite { test("date casting") { val d = Date.valueOf("1970-01-01") - checkEvaluation(Cast(d, ShortType), null) - checkEvaluation(Cast(d, IntegerType), null) - checkEvaluation(Cast(d, LongType), null) - checkEvaluation(Cast(d, FloatType), null) - checkEvaluation(Cast(d, DoubleType), null) - checkEvaluation(Cast(d, DecimalType.Unlimited), null) - checkEvaluation(Cast(d, DecimalType(10, 2)), null) - checkEvaluation(Cast(d, StringType), "1970-01-01") - checkEvaluation(Cast(Cast(d, TimestampType), StringType), "1970-01-01 00:00:00") + checkEvaluation(Cast(Literal(d), ShortType), null) + checkEvaluation(Cast(Literal(d), IntegerType), null) + checkEvaluation(Cast(Literal(d), LongType), null) + checkEvaluation(Cast(Literal(d), FloatType), null) + checkEvaluation(Cast(Literal(d), DoubleType), null) + checkEvaluation(Cast(Literal(d), DecimalType.Unlimited), null) + checkEvaluation(Cast(Literal(d), DecimalType(10, 2)), null) + checkEvaluation(Cast(Literal(d), StringType), "1970-01-01") + checkEvaluation(Cast(Cast(Literal(d), TimestampType), StringType), "1970-01-01 00:00:00") } test("timestamp casting") { val millis = 15 * 1000 + 2 val seconds = millis * 1000 + 2 val ts = new Timestamp(millis) - val ts1 = new Timestamp(15 * 1000) // a timestamp without the milliseconds part val tss = new Timestamp(seconds) checkEvaluation(Cast(ts, ShortType), 15) checkEvaluation(Cast(ts, IntegerType), 15) 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 c147be9f6b1ae..7bcd6687d11a1 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 @@ -106,7 +106,7 @@ class DataTypeSuite extends FunSuite { checkDefaultSize(DoubleType, 8) checkDefaultSize(DecimalType(10, 5), 4096) checkDefaultSize(DecimalType.Unlimited, 4096) - checkDefaultSize(DateType, 8) + checkDefaultSize(DateType, 4) checkDefaultSize(TimestampType, 8) checkDefaultSize(StringType, 4096) checkDefaultSize(BinaryType, 4096) 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 391b3dae5c8ce..cad0667b46435 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.columnar -import java.sql.{Date, Timestamp} +import java.sql.Timestamp import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.{AttributeMap, Attribute, AttributeReference} @@ -215,22 +215,7 @@ private[sql] class StringColumnStats extends ColumnStats { def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } -private[sql] class DateColumnStats extends ColumnStats { - protected var upper: Date = null - protected var lower: Date = null - - override def gatherStats(row: Row, ordinal: Int) { - super.gatherStats(row, ordinal) - if (!row.isNullAt(ordinal)) { - val value = row(ordinal).asInstanceOf[Date] - if (upper == null || value.compareTo(upper) > 0) upper = value - if (lower == null || value.compareTo(lower) < 0) lower = value - sizeInBytes += DATE.defaultSize - } - } - - def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) -} +private[sql] class DateColumnStats extends IntColumnStats private[sql] class TimestampColumnStats extends ColumnStats { protected var upper: Timestamp = null 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 fcf2faa0914c0..db5bc0de363c7 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 @@ -335,21 +335,20 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { } } -private[sql] object DATE extends NativeColumnType(DateType, 8, 8) { +private[sql] object DATE extends NativeColumnType(DateType, 8, 4) { override def extract(buffer: ByteBuffer) = { - val date = new Date(buffer.getLong()) - date + buffer.getInt } - override def append(v: Date, buffer: ByteBuffer): Unit = { - buffer.putLong(v.getTime) + override def append(v: Int, buffer: ByteBuffer): Unit = { + buffer.putInt(v) } override def getField(row: Row, ordinal: Int) = { - row(ordinal).asInstanceOf[Date] + row(ordinal).asInstanceOf[Int] } - override def setField(row: MutableRow, ordinal: Int, value: Date): Unit = { + def setField(row: MutableRow, ordinal: Int, value: Int): Unit = { row(ordinal) = value } } 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 b85021acc9d4c..3a2f8d75dac5e 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 @@ -135,6 +135,8 @@ object EvaluatePython { case (ud, udt: UserDefinedType[_]) => toJava(udt.serialize(ud), udt.sqlType) + case (date: Int, DateType) => DateUtils.toJavaDate(date) + // Pyrolite can handle Timestamp and Decimal case (other, _) => other } @@ -171,7 +173,7 @@ object EvaluatePython { }): Row case (c: java.util.Calendar, DateType) => - new java.sql.Date(c.getTime().getTime()) + DateUtils.fromJavaDate(new java.sql.Date(c.getTime().getTime())) case (c: java.util.Calendar, TimestampType) => new java.sql.Timestamp(c.getTime().getTime()) 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 9171939f7e8f7..33ce71b51b213 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 @@ -377,10 +377,12 @@ private[sql] object JsonRDD extends Logging { } } - private def toDate(value: Any): Date = { + private def toDate(value: Any): Int = { value match { // only support string as date - case value: java.lang.String => new Date(DataTypeConversions.stringToTime(value).getTime) + case value: java.lang.String => + DateUtils.millisToDays(DataTypeConversions.stringToTime(value).getTime) + case value: java.sql.Date => DateUtils.fromJavaDate(value) } } 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 e18ba287e8683..0501b47f080d7 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 @@ -38,7 +38,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { var origZone: TimeZone = _ override protected def beforeAll() { origZone = TimeZone.getDefault - TimeZone.setDefault(TimeZone.getTimeZone("UTC")) + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) } override protected def afterAll() { @@ -143,26 +143,26 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { test("SPARK-3173 Timestamp support in the parser") { checkAnswer(sql( - "SELECT time FROM timestamps WHERE time=CAST('1970-01-01 00:00:00.001' AS TIMESTAMP)"), - Row(java.sql.Timestamp.valueOf("1970-01-01 00:00:00.001"))) + "SELECT time FROM timestamps WHERE time=CAST('1969-12-31 16:00:00.001' AS TIMESTAMP)"), + Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.001"))) checkAnswer(sql( - "SELECT time FROM timestamps WHERE time='1970-01-01 00:00:00.001'"), - Row(java.sql.Timestamp.valueOf("1970-01-01 00:00:00.001"))) + "SELECT time FROM timestamps WHERE time='1969-12-31 16:00:00.001'"), + Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.001"))) checkAnswer(sql( - "SELECT time FROM timestamps WHERE '1970-01-01 00:00:00.001'=time"), - Row(java.sql.Timestamp.valueOf("1970-01-01 00:00:00.001"))) + "SELECT time FROM timestamps WHERE '1969-12-31 16:00:00.001'=time"), + Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.001"))) checkAnswer(sql( - """SELECT time FROM timestamps WHERE time<'1970-01-01 00:00:00.003' - AND time>'1970-01-01 00:00:00.001'"""), - Row(java.sql.Timestamp.valueOf("1970-01-01 00:00:00.002"))) + """SELECT time FROM timestamps WHERE time<'1969-12-31 16:00:00.003' + AND time>'1969-12-31 16:00:00.001'"""), + Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.002"))) checkAnswer(sql( - "SELECT time FROM timestamps WHERE time IN ('1970-01-01 00:00:00.001','1970-01-01 00:00:00.002')"), - Seq(Row(java.sql.Timestamp.valueOf("1970-01-01 00:00:00.001")), - Row(java.sql.Timestamp.valueOf("1970-01-01 00:00:00.002")))) + "SELECT time FROM timestamps WHERE time IN ('1969-12-31 16:00:00.001','1969-12-31 16:00:00.002')"), + 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")))) checkAnswer(sql( "SELECT time FROM timestamps WHERE time='123'"), @@ -296,6 +296,13 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { mapData.collect().take(1).map(Row.fromTuple).toSeq) } + test("date row") { + checkAnswer(sql( + """select cast("2015-01-28" as date) from testData limit 1"""), + Row(java.sql.Date.valueOf("2015-01-28")) + ) + } + 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"), 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 a015884bae282..f26fcc0385b68 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 @@ -83,7 +83,8 @@ 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(12345), new Timestamp(12345), Seq(1,2,3))) + new java.math.BigDecimal(1), new Date(70, 0, 1), // This is 1970-01-01 + new Timestamp(12345), Seq(1,2,3))) } test("query case class RDD with nulls") { 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 be2b34de077c9..581fccf8ee613 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 @@ -30,7 +30,7 @@ class ColumnStatsSuite extends FunSuite { testColumnStats(classOf[FloatColumnStats], FLOAT, Row(Float.MaxValue, Float.MinValue, 0)) testColumnStats(classOf[DoubleColumnStats], DOUBLE, Row(Double.MaxValue, Double.MinValue, 0)) testColumnStats(classOf[StringColumnStats], STRING, Row(null, null, 0)) - testColumnStats(classOf[DateColumnStats], DATE, Row(null, null, 0)) + testColumnStats(classOf[DateColumnStats], DATE, Row(Int.MaxValue, Int.MinValue, 0)) testColumnStats(classOf[TimestampColumnStats], TIMESTAMP, Row(null, null, 0)) def testColumnStats[T <: NativeType, U <: ColumnStats]( 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 87e608a8853dc..9ce845912f1c7 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer -import java.sql.{Date, Timestamp} +import java.sql.Timestamp import org.scalatest.FunSuite @@ -34,7 +34,7 @@ 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 -> 8, TIMESTAMP -> 12, BINARY -> 16, GENERIC -> 16) + STRING -> 8, DATE -> 4, TIMESTAMP -> 12, BINARY -> 16, GENERIC -> 16) checks.foreach { case (columnType, expectedSize) => assertResult(expectedSize, s"Wrong defaultSize for $columnType") { @@ -64,7 +64,7 @@ class ColumnTypeSuite extends FunSuite with Logging { checkActualSize(FLOAT, Float.MaxValue, 4) checkActualSize(BOOLEAN, true, 1) checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) - checkActualSize(DATE, new Date(0L), 8) + checkActualSize(DATE, 0, 4) checkActualSize(TIMESTAMP, new Timestamp(0L), 12) val binary = Array.fill[Byte](4)(0: 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 f941465fa3e35..60ed28cc97bf1 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 @@ -17,11 +17,11 @@ package org.apache.spark.sql.columnar +import java.sql.Timestamp + import scala.collection.immutable.HashSet import scala.util.Random -import java.sql.{Date, Timestamp} - import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.types.{DataType, NativeType} @@ -50,7 +50,7 @@ object ColumnarTestUtils { case STRING => Random.nextString(Random.nextInt(32)) case BOOLEAN => Random.nextBoolean() case BINARY => randomBytes(Random.nextInt(32)) - case DATE => new Date(Random.nextLong()) + 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/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index cb615388da0c7..1396c6b7246d1 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 @@ -67,14 +67,15 @@ class JsonSuite extends QueryTest { checkTypePromotion(Timestamp.valueOf(strTime), enforceCorrectType(strTime, TimestampType)) val strDate = "2014-10-15" - checkTypePromotion(Date.valueOf(strDate), enforceCorrectType(strDate, DateType)) + checkTypePromotion( + DateUtils.fromJavaDate(Date.valueOf(strDate)), enforceCorrectType(strDate, DateType)) val ISO8601Time1 = "1970-01-01T01:00:01.0Z" checkTypePromotion(new Timestamp(3601000), enforceCorrectType(ISO8601Time1, TimestampType)) - checkTypePromotion(new Date(3601000), enforceCorrectType(ISO8601Time1, DateType)) + checkTypePromotion(DateUtils.millisToDays(3601000), enforceCorrectType(ISO8601Time1, DateType)) val ISO8601Time2 = "1970-01-01T02:00:01-01:00" checkTypePromotion(new Timestamp(10801000), enforceCorrectType(ISO8601Time2, TimestampType)) - checkTypePromotion(new Date(10801000), enforceCorrectType(ISO8601Time2, DateType)) + checkTypePromotion(DateUtils.millisToDays(10801000), enforceCorrectType(ISO8601Time2, DateType)) } test("Get compatible type") { 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 0d934620aca09..a6266f611c219 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 @@ -357,6 +357,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "database_drop", "database_location", "database_properties", + "date_1", "date_2", "date_3", "date_4", 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 922e61f0be12f..1921bf6e5e1a6 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive import java.io.{BufferedReader, InputStreamReader, PrintStream} -import java.sql.{Date, Timestamp} +import java.sql.Timestamp import scala.collection.JavaConversions._ import scala.language.implicitConversions @@ -478,7 +478,7 @@ private object HiveContext { toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) }.toSeq.sorted.mkString("{", ",", "}") case (null, _) => "NULL" - case (d: Date, DateType) => new DateWritable(d).toString + case (d: Int, DateType) => new DateWritable(d).toString case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8") case (decimal: java.math.BigDecimal, DecimalType()) => 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 82dba99900df9..4afa2e71d77cc 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 @@ -267,7 +267,8 @@ private[hive] trait HiveInspectors { val temp = new Array[Byte](writable.getLength) System.arraycopy(writable.getBytes, 0, temp, 0, temp.length) temp - case poi: WritableConstantDateObjectInspector => poi.getWritableConstantValue.get() + case poi: WritableConstantDateObjectInspector => + DateUtils.fromJavaDate(poi.getWritableConstantValue.get()) case mi: StandardConstantMapObjectInspector => // take the value from the map inspector object, rather than the input data mi.getWritableConstantValue.map { case (k, v) => @@ -304,7 +305,8 @@ private[hive] trait HiveInspectors { System.arraycopy(bw.getBytes(), 0, result, 0, bw.getLength()) result case x: DateObjectInspector if x.preferWritable() => - x.getPrimitiveWritableObject(data).get() + DateUtils.fromJavaDate(x.getPrimitiveWritableObject(data).get()) + case x: DateObjectInspector => DateUtils.fromJavaDate(x.getPrimitiveJavaObject(data)) // org.apache.hadoop.hive.serde2.io.TimestampWritable.set will reset current time object // if next timestamp is null, so Timestamp object is cloned case x: TimestampObjectInspector if x.preferWritable() => @@ -343,6 +345,9 @@ private[hive] trait HiveInspectors { case _: JavaHiveDecimalObjectInspector => (o: Any) => HiveShim.createDecimal(o.asInstanceOf[Decimal].toJavaBigDecimal) + case _: JavaDateObjectInspector => + (o: Any) => DateUtils.toJavaDate(o.asInstanceOf[Int]) + case soi: StandardStructObjectInspector => val wrappers = soi.getAllStructFieldRefs.map(ref => wrapperFor(ref.getFieldObjectInspector)) (o: Any) => { @@ -426,7 +431,7 @@ private[hive] trait HiveInspectors { case _: BinaryObjectInspector if x.preferWritable() => HiveShim.getBinaryWritable(a) case _: BinaryObjectInspector => a.asInstanceOf[Array[Byte]] case _: DateObjectInspector if x.preferWritable() => HiveShim.getDateWritable(a) - case _: DateObjectInspector => a.asInstanceOf[java.sql.Date] + case _: DateObjectInspector => DateUtils.toJavaDate(a.asInstanceOf[Int]) case _: TimestampObjectInspector if x.preferWritable() => HiveShim.getTimestampWritable(a) case _: TimestampObjectInspector => a.asInstanceOf[java.sql.Timestamp] } 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 c368715f7c6f5..effaa5a443512 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 @@ -34,6 +34,7 @@ import org.apache.spark.SerializableWritable 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 /** * A trait for subclasses that handle table scans. @@ -306,7 +307,7 @@ private[hive] object HadoopTableReader extends HiveInspectors { row.update(ordinal, oi.getPrimitiveJavaObject(value).clone()) case oi: DateObjectInspector => (value: Any, row: MutableRow, ordinal: Int) => - row.update(ordinal, oi.getPrimitiveJavaObject(value)) + row.update(ordinal, DateUtils.fromJavaDate(oi.getPrimitiveJavaObject(value))) case oi: BinaryObjectInspector => (value: Any, row: MutableRow, ordinal: Int) => row.update(ordinal, oi.getPrimitiveJavaObject(value)) diff --git a/sql/hive/src/test/resources/golden/Date cast-0-a7cd69b80c77a771a2c955db666be53d b/sql/hive/src/test/resources/golden/Date cast-0-a7cd69b80c77a771a2c955db666be53d new file mode 100644 index 0000000000000..98da82fa89386 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Date cast-0-a7cd69b80c77a771a2c955db666be53d @@ -0,0 +1 @@ +1970-01-01 1970-01-01 1969-12-31 16:00:00 1969-12-31 16:00:00 1970-01-01 00:00:00 diff --git a/sql/hive/src/test/resources/golden/Date comparison test 1-0-bde89be08a12361073ff658fef768b7e b/sql/hive/src/test/resources/golden/Date comparison test 1-0-bde89be08a12361073ff658fef768b7e new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Date comparison test 1-0-bde89be08a12361073ff658fef768b7e @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/Date comparison test 2-0-dc1b267f1d79d49e6675afe4fd2a34a5 b/sql/hive/src/test/resources/golden/Date comparison test 2-0-dc1b267f1d79d49e6675afe4fd2a34a5 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Date comparison test 2-0-dc1b267f1d79d49e6675afe4fd2a34a5 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/date_1-0-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/date_1-0-50131c0ba7b7a6b65c789a5a8497bada new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-0-50131c0ba7b7a6b65c789a5a8497bada @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/date_1-0-23edf29bf7376c70d5ecf12720f4b1eb b/sql/hive/src/test/resources/golden/date_1-1-23edf29bf7376c70d5ecf12720f4b1eb similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-0-23edf29bf7376c70d5ecf12720f4b1eb rename to sql/hive/src/test/resources/golden/date_1-1-23edf29bf7376c70d5ecf12720f4b1eb diff --git a/sql/hive/src/test/resources/golden/date_1-3-df16364a220ff96a6ea1cd478cbc1d0b b/sql/hive/src/test/resources/golden/date_1-10-df16364a220ff96a6ea1cd478cbc1d0b similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-3-df16364a220ff96a6ea1cd478cbc1d0b rename to sql/hive/src/test/resources/golden/date_1-10-df16364a220ff96a6ea1cd478cbc1d0b diff --git a/sql/hive/src/test/resources/golden/date_1-10-d964bec7e5632091ab5cb6f6786dbbf9 b/sql/hive/src/test/resources/golden/date_1-11-d964bec7e5632091ab5cb6f6786dbbf9 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-10-d964bec7e5632091ab5cb6f6786dbbf9 rename to sql/hive/src/test/resources/golden/date_1-11-d964bec7e5632091ab5cb6f6786dbbf9 diff --git a/sql/hive/src/test/resources/golden/date_1-11-480c5f024a28232b7857be327c992509 b/sql/hive/src/test/resources/golden/date_1-12-480c5f024a28232b7857be327c992509 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-11-480c5f024a28232b7857be327c992509 rename to sql/hive/src/test/resources/golden/date_1-12-480c5f024a28232b7857be327c992509 diff --git a/sql/hive/src/test/resources/golden/date_1-12-4c0ed7fcb75770d8790575b586bf14f4 b/sql/hive/src/test/resources/golden/date_1-13-4c0ed7fcb75770d8790575b586bf14f4 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-12-4c0ed7fcb75770d8790575b586bf14f4 rename to sql/hive/src/test/resources/golden/date_1-13-4c0ed7fcb75770d8790575b586bf14f4 diff --git a/sql/hive/src/test/resources/golden/date_1-13-44fc74c1993062c0a9522199ff27fea b/sql/hive/src/test/resources/golden/date_1-14-44fc74c1993062c0a9522199ff27fea similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-13-44fc74c1993062c0a9522199ff27fea rename to sql/hive/src/test/resources/golden/date_1-14-44fc74c1993062c0a9522199ff27fea diff --git a/sql/hive/src/test/resources/golden/date_1-14-4855a66124b16d1d0d003235995ac06b b/sql/hive/src/test/resources/golden/date_1-15-4855a66124b16d1d0d003235995ac06b similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-14-4855a66124b16d1d0d003235995ac06b rename to sql/hive/src/test/resources/golden/date_1-15-4855a66124b16d1d0d003235995ac06b diff --git a/sql/hive/src/test/resources/golden/date_1-15-8bc190dba0f641840b5e1e198a14c55b b/sql/hive/src/test/resources/golden/date_1-16-8bc190dba0f641840b5e1e198a14c55b similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-15-8bc190dba0f641840b5e1e198a14c55b rename to sql/hive/src/test/resources/golden/date_1-16-8bc190dba0f641840b5e1e198a14c55b diff --git a/sql/hive/src/test/resources/golden/date_1-1-4ebe3571c13a8b0c03096fbd972b7f1b b/sql/hive/src/test/resources/golden/date_1-17-23edf29bf7376c70d5ecf12720f4b1eb similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-1-4ebe3571c13a8b0c03096fbd972b7f1b rename to sql/hive/src/test/resources/golden/date_1-17-23edf29bf7376c70d5ecf12720f4b1eb diff --git a/sql/hive/src/test/resources/golden/date_1-16-23edf29bf7376c70d5ecf12720f4b1eb b/sql/hive/src/test/resources/golden/date_1-2-4ebe3571c13a8b0c03096fbd972b7f1b similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-16-23edf29bf7376c70d5ecf12720f4b1eb rename to sql/hive/src/test/resources/golden/date_1-2-4ebe3571c13a8b0c03096fbd972b7f1b diff --git a/sql/hive/src/test/resources/golden/date_1-2-abdce0c0d14d3fc7441b7c134b02f99a b/sql/hive/src/test/resources/golden/date_1-3-26b5c291400dfde455b3c1b878b71d0 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-2-abdce0c0d14d3fc7441b7c134b02f99a rename to sql/hive/src/test/resources/golden/date_1-3-26b5c291400dfde455b3c1b878b71d0 diff --git a/sql/hive/src/test/resources/golden/date_1-6-df16364a220ff96a6ea1cd478cbc1d0b b/sql/hive/src/test/resources/golden/date_1-4-df16364a220ff96a6ea1cd478cbc1d0b similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-6-df16364a220ff96a6ea1cd478cbc1d0b rename to sql/hive/src/test/resources/golden/date_1-4-df16364a220ff96a6ea1cd478cbc1d0b diff --git a/sql/hive/src/test/resources/golden/date_1-4-d964bec7e5632091ab5cb6f6786dbbf9 b/sql/hive/src/test/resources/golden/date_1-5-d964bec7e5632091ab5cb6f6786dbbf9 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-4-d964bec7e5632091ab5cb6f6786dbbf9 rename to sql/hive/src/test/resources/golden/date_1-5-d964bec7e5632091ab5cb6f6786dbbf9 diff --git a/sql/hive/src/test/resources/golden/date_1-5-5e70fc74158fbfca38134174360de12d b/sql/hive/src/test/resources/golden/date_1-6-559d01fb0b42c42f0c4927fa0f9deac4 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-5-5e70fc74158fbfca38134174360de12d rename to sql/hive/src/test/resources/golden/date_1-6-559d01fb0b42c42f0c4927fa0f9deac4 diff --git a/sql/hive/src/test/resources/golden/date_1-9-df16364a220ff96a6ea1cd478cbc1d0b b/sql/hive/src/test/resources/golden/date_1-7-df16364a220ff96a6ea1cd478cbc1d0b similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-9-df16364a220ff96a6ea1cd478cbc1d0b rename to sql/hive/src/test/resources/golden/date_1-7-df16364a220ff96a6ea1cd478cbc1d0b diff --git a/sql/hive/src/test/resources/golden/date_1-7-d964bec7e5632091ab5cb6f6786dbbf9 b/sql/hive/src/test/resources/golden/date_1-8-d964bec7e5632091ab5cb6f6786dbbf9 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-7-d964bec7e5632091ab5cb6f6786dbbf9 rename to sql/hive/src/test/resources/golden/date_1-8-d964bec7e5632091ab5cb6f6786dbbf9 diff --git a/sql/hive/src/test/resources/golden/date_1-8-1d5c58095cd52ea539d869f2ab1ab67d b/sql/hive/src/test/resources/golden/date_1-9-8306558e0eabe936ac33dabaaa17fea4 similarity index 100% rename from sql/hive/src/test/resources/golden/date_1-8-1d5c58095cd52ea539d869f2ab1ab67d rename to sql/hive/src/test/resources/golden/date_1-9-8306558e0eabe936ac33dabaaa17fea4 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 2d3ff680125ad..09bbd5c867e4e 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 @@ -18,7 +18,6 @@ package org.apache.spark.sql.hive import java.util -import java.sql.Date import java.util.{Locale, TimeZone} import org.apache.hadoop.hive.ql.udf.UDAFPercentile @@ -76,7 +75,7 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { Literal(0.asInstanceOf[Float]) :: Literal(0.asInstanceOf[Double]) :: Literal("0") :: - Literal(new Date(2014, 9, 23)) :: + Literal(new java.sql.Date(114, 8, 23)) :: Literal(Decimal(BigDecimal(123.123))) :: Literal(new java.sql.Timestamp(123123)) :: Literal(Array[Byte](1,2,3)) :: @@ -143,7 +142,6 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { 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) } - case (r1: Date, r2: Date) => assert(r1.compareTo(r2) === 0) case (r1, r2) => assert(r1 === r2) } } 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 8e84d279fe119..82efadb28e890 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 @@ -253,8 +253,30 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("Cast Timestamp to Timestamp in UDF", """ - | SELECT DATEDIFF(CAST(value AS timestamp), CAST('2002-03-21 00:00:00' AS timestamp)) - | FROM src LIMIT 1 + | SELECT DATEDIFF(CAST(value AS timestamp), CAST('2002-03-21 00:00:00' AS timestamp)) + | FROM src LIMIT 1 + """.stripMargin) + + createQueryTest("Date comparison test 1", + """ + | SELECT + | CAST(CAST('1970-01-01 22:00:00' AS timestamp) AS date) == + | CAST(CAST('1970-01-01 23:00:00' AS timestamp) AS date) + | FROM src LIMIT 1 + """.stripMargin) + + createQueryTest("Date comparison test 2", + "SELECT CAST(CAST(0 AS timestamp) AS date) > CAST(0 AS timestamp) FROM src LIMIT 1") + + createQueryTest("Date cast", + """ + | SELECT + | CAST(CAST(0 AS timestamp) AS date), + | CAST(CAST(CAST(0 AS timestamp) AS date) AS string), + | CAST(0 AS timestamp), + | CAST(CAST(0 AS timestamp) AS string), + | CAST(CAST(CAST('1970-01-01 23:00:00' AS timestamp) AS date) AS timestamp) + | FROM src LIMIT 1 """.stripMargin) createQueryTest("Simple Average", 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 254919e8f6fdc..b5a0754ff61f9 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 @@ -160,7 +160,7 @@ private[hive] object HiveShim { if (value == null) null else new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]]) def getDateWritable(value: Any): hiveIo.DateWritable = - if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[java.sql.Date]) + if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[Int]) def getTimestampWritable(value: Any): hiveIo.TimestampWritable = if (value == null) { 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 45ca59ae56a38..e4c1809c8bb21 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 @@ -263,7 +263,7 @@ private[hive] object HiveShim { } def getDateWritable(value: Any): hiveIo.DateWritable = - if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[java.sql.Date]) + if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[Int]) def getTimestampWritable(value: Any): hiveIo.TimestampWritable = if (value == null) { From 681f9df47ff40f7b0d9175d835e9758d33a13a06 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 3 Feb 2015 13:46:02 -0800 Subject: [PATCH 049/578] [SPARK-5153][Streaming][Test] Increased timeout to deal with flaky KafkaStreamSuite Timeout increased to allow overloaded Jenkins to cope with delay in topic creation. Author: Tathagata Das Closes #4342 from tdas/SPARK-5153 and squashes the following commits: dc42762 [Tathagata Das] Increased timeout to deal with delays in overloaded Jenkins. --- .../org/apache/spark/streaming/kafka/KafkaStreamSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 b19c053ebfc44..0817c56d8f39f 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 @@ -164,7 +164,7 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Loggin } private def waitUntilMetadataIsPropagated(topic: String, partition: Int) { - eventually(timeout(1000 milliseconds), interval(100 milliseconds)) { + eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { assert( server.apis.leaderCache.keySet.contains(TopicAndPartition(topic, partition)), s"Partition [$topic, $partition] metadata not propagated after timeout" From 1e8b5394b44a0d3b36f64f10576c3ae3b977810c Mon Sep 17 00:00:00 2001 From: Jesper Lundgren Date: Tue, 3 Feb 2015 14:53:39 -0800 Subject: [PATCH 050/578] [STREAMING] SPARK-4986 Wait for receivers to deregister and receiver job to terminate A slow receiver might not have enough time to shutdown cleanly even when graceful shutdown is used. This PR extends graceful waiting to make sure all receivers have deregistered and that the receiver job has terminated. Author: Jesper Lundgren Closes #4338 from cleaton/stopreceivers and squashes the following commits: a9cf223 [Jesper Lundgren] remove cleaner.ttl config f969b6e [Jesper Lundgren] fix inversed logic in unit test 3d0bd35 [Jesper Lundgren] switch boleans to match running status instead of terminated 9a9ff88 [Jesper Lundgren] wait for receivers to shutdown and receiver job to terminate d179372 [Jesper Lundgren] Add graceful shutdown unit test covering slow receiver onStop --- .../streaming/scheduler/JobScheduler.scala | 2 +- .../streaming/scheduler/ReceiverTracker.scala | 19 +++++- .../streaming/StreamingContextSuite.scala | 58 +++++++++++++++++++ 3 files changed, 75 insertions(+), 4 deletions(-) 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 0e0f5bd3b9db4..b3ffc71904c76 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 @@ -73,7 +73,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { logDebug("Stopping JobScheduler") // First, stop receiving - receiverTracker.stop() + receiverTracker.stop(processAllReceivedData) // Second, stop generating jobs. If it has to process all received data, // then this will wait for all the processing through JobScheduler to be over. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 4f998869731ed..00456ab2a0c92 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 @@ -86,10 +86,10 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } /** Stop the receiver execution thread. */ - def stop() = synchronized { + def stop(graceful: Boolean) = synchronized { if (!receiverInputStreams.isEmpty && actor != null) { // First, stop the receivers - if (!skipReceiverLaunch) receiverExecutor.stop() + if (!skipReceiverLaunch) receiverExecutor.stop(graceful) // Finally, stop the actor ssc.env.actorSystem.stop(actor) @@ -218,6 +218,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false /** This thread class runs all the receivers on the cluster. */ class ReceiverLauncher { @transient val env = ssc.env + @volatile @transient private var running = false @transient val thread = new Thread() { override def run() { try { @@ -233,7 +234,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false thread.start() } - def stop() { + def stop(graceful: Boolean) { // Send the stop signal to all the receivers stopReceivers() @@ -241,6 +242,16 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false // That is, for the receivers to quit gracefully. thread.join(10000) + if (graceful) { + val pollTime = 100 + def done = { receiverInfo.isEmpty && !running } + logInfo("Waiting for receiver job to terminate gracefully") + while(!done) { + Thread.sleep(pollTime) + } + logInfo("Waited for receiver job to terminate gracefully") + } + // Check if all the receivers have been deregistered or not if (!receiverInfo.isEmpty) { logWarning("All of the receivers have not deregistered, " + receiverInfo) @@ -295,7 +306,9 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false // Distribute the receivers and start them logInfo("Starting " + receivers.length + " receivers") + running = true ssc.sparkContext.runJob(tempRDD, ssc.sparkContext.clean(startReceiver)) + running = false logInfo("All of the receivers have been terminated") } 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 9f352bdcb0893..0b5af25e0f7cc 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -205,6 +205,32 @@ 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") + sc = new SparkContext(conf) + logInfo("==================================\n\n\n") + ssc = new StreamingContext(sc, Milliseconds(100)) + var runningCount = 0 + SlowTestReceiver.receivedAllRecords = false + //Create test receiver that sleeps in onStop() + val totalNumRecords = 15 + val recordsPerSecond = 1 + val input = ssc.receiverStream(new SlowTestReceiver(totalNumRecords, recordsPerSecond)) + input.count().foreachRDD { rdd => + val count = rdd.first() + runningCount += count.toInt + logInfo("Count = " + count + ", Running count = " + runningCount) + } + ssc.start() + ssc.awaitTermination(500) + ssc.stop(stopSparkContext = false, stopGracefully = true) + logInfo("Running count = " + runningCount) + assert(runningCount > 0) + assert(runningCount == totalNumRecords) + Thread.sleep(100) + } + test("awaitTermination") { ssc = new StreamingContext(master, appName, batchDuration) val inputStream = addInputStream(ssc) @@ -319,6 +345,38 @@ object TestReceiver { val counter = new AtomicInteger(1) } +/** 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 { + + var receivingThreadOption: Option[Thread] = None + + def onStart() { + val thread = new Thread() { + override def run() { + logInfo("Receiving started") + for(i <- 1 to totalRecords) { + Thread.sleep(1000 / recordsPerSecond) + store(i) + } + SlowTestReceiver.receivedAllRecords = true + logInfo(s"Received all $totalRecords records") + } + } + receivingThreadOption = Some(thread) + thread.start() + } + + def onStop() { + // Simulate slow receiver by waiting for all records to be produced + while(!SlowTestReceiver.receivedAllRecords) Thread.sleep(100) + // no cleanup to be done, the receiving thread should stop on it own + } +} + +object SlowTestReceiver { + var receivedAllRecords = false +} + /** Streaming application for testing DStream and RDD creation sites */ package object testPackage extends Assertions { def test() { From 068c0e2ee05ee8b133c2dc26b8fa094ab2712d45 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 3 Feb 2015 16:01:56 -0800 Subject: [PATCH 051/578] [SPARK-5554] [SQL] [PySpark] add more tests for DataFrame Python API Add more tests and docs for DataFrame Python API, improve test coverage, fix bugs. Author: Davies Liu Closes #4331 from davies/fix_df and squashes the following commits: dd9919f [Davies Liu] fix tests 467332c [Davies Liu] support string in cast() 83c92fe [Davies Liu] address comments c052f6f [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_df 8dd19a9 [Davies Liu] fix tests in python 2.6 35ccb9f [Davies Liu] fix build 78ebcfa [Davies Liu] add sql_test.py in run_tests 9ab78b4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_df 6040ba7 [Davies Liu] fix docs 3ab2661 [Davies Liu] add more tests for DataFrame --- python/pyspark/sql.py | 467 +++++++++++------- python/pyspark/sql_tests.py | 299 +++++++++++ python/pyspark/tests.py | 261 ---------- python/run-tests | 1 + .../scala/org/apache/spark/sql/Column.scala | 38 +- .../spark/sql/test/ExamplePointUDT.scala | 2 +- 6 files changed, 586 insertions(+), 482 deletions(-) create mode 100644 python/pyspark/sql_tests.py diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 32bff0c7e8c55..268c7ef97cffc 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -62,7 +62,7 @@ "StringType", "BinaryType", "BooleanType", "DateType", "TimestampType", "DecimalType", "DoubleType", "FloatType", "ByteType", "IntegerType", "LongType", "ShortType", "ArrayType", "MapType", "StructField", "StructType", - "SQLContext", "HiveContext", "DataFrame", "GroupedDataFrame", "Column", "Row", + "SQLContext", "HiveContext", "DataFrame", "GroupedDataFrame", "Column", "Row", "Dsl", "SchemaRDD"] @@ -1804,7 +1804,7 @@ class DataFrame(object): people = sqlContext.parquetFile("...") Once created, it can be manipulated using the various domain-specific-language - (DSL) functions defined in: [[DataFrame]], [[Column]]. + (DSL) functions defined in: :class:`DataFrame`, :class:`Column`. To select a column from the data frame, use the apply method:: @@ -1835,8 +1835,10 @@ def __init__(self, jdf, sql_ctx): @property def rdd(self): - """Return the content of the :class:`DataFrame` as an :class:`RDD` - of :class:`Row`s. """ + """ + 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())) @@ -1850,18 +1852,6 @@ def applySchema(it): return self._lazy_rdd - def limit(self, num): - """Limit the result count to the number specified. - - >>> df = sqlCtx.inferSchema(rdd) - >>> df.limit(2).collect() - [Row(field1=1, field2=u'row1'), Row(field1=2, field2=u'row2')] - >>> df.limit(0).collect() - [] - """ - jdf = self._jdf.limit(num) - return DataFrame(jdf, self.sql_ctx) - def toJSON(self, use_unicode=False): """Convert a DataFrame into a MappedRDD of JSON documents; one document per row. @@ -1886,7 +1876,6 @@ def saveAsParquetFile(self, path): >>> import tempfile, shutil >>> parquetFile = tempfile.mkdtemp() >>> shutil.rmtree(parquetFile) - >>> df = sqlCtx.inferSchema(rdd) >>> df.saveAsParquetFile(parquetFile) >>> df2 = sqlCtx.parquetFile(parquetFile) >>> sorted(df2.collect()) == sorted(df.collect()) @@ -1900,9 +1889,8 @@ def registerTempTable(self, name): The lifetime of this temporary table is tied to the L{SQLContext} that was used to create this DataFrame. - >>> df = sqlCtx.inferSchema(rdd) - >>> df.registerTempTable("test") - >>> df2 = sqlCtx.sql("select * from test") + >>> df.registerTempTable("people") + >>> df2 = sqlCtx.sql("select * from people") >>> sorted(df.collect()) == sorted(df2.collect()) True """ @@ -1926,11 +1914,22 @@ def saveAsTable(self, tableName): def schema(self): """Returns the schema of this DataFrame (represented by - a L{StructType}).""" + 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.""" + """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): @@ -1940,11 +1939,8 @@ def count(self): leverages the query optimizer to compute the count on the DataFrame, which supports features such as filter pushdown. - >>> df = sqlCtx.inferSchema(rdd) >>> df.count() - 3L - >>> df.count() == df.map(lambda x: x).count() - True + 2L """ return self._jdf.count() @@ -1954,13 +1950,11 @@ def collect(self): Each object in the list is a Row, the fields can be accessed as attributes. - >>> df = sqlCtx.inferSchema(rdd) >>> df.collect() - [Row(field1=1, field2=u'row1'), ..., Row(field1=3, field2=u'row3')] + [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] """ with SCCallSiteSync(self._sc) as css: bytesInJava = self._jdf.javaToPython().collect().iterator() - cls = _create_cls(self.schema()) tempFile = NamedTemporaryFile(delete=False, dir=self._sc._temp_dir) tempFile.close() self._sc._writeToFile(bytesInJava, tempFile.name) @@ -1968,23 +1962,37 @@ 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()) 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 = sqlCtx.inferSchema(rdd) >>> df.take(2) - [Row(field1=1, field2=u'row1'), Row(field1=2, field2=u'row2')] + [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) @@ -2067,140 +2075,167 @@ def sample(self, withReplacement, fraction, seed=None): @property def dtypes(self): """Return all column names and their data types as a list. + + >>> df.dtypes + [(u'age', 'IntegerType'), (u'name', 'StringType')] """ return [(f.name, str(f.dataType)) 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 show(self): - raise NotImplemented - 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`:: - df1.join(df2, df1.key == df2.key, "outer") - :param other: Right side of the join :param joinExprs: Join expression - :param joinType: One of `inner`, `outer`, `left_outer`, `right_outer`, - `semijoin`. + :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 joinType is None: - if joinExprs is None: - jdf = self._jdf.join(other._jdf) - else: - jdf = self._jdf.join(other._jdf, joinExprs) + + if joinExprs is None: + jdf = self._jdf.join(other._jdf) else: - jdf = self._jdf.join(other._jdf, joinExprs, joinType) + 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 [[DataFrame]] sorted by the specified column, - in ascending column. + """ 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") - for i, c in enumerate(cols): - if isinstance(c, basestring): - cols[i] = Column(c) - jcols = [c._jc for c in cols] - jdf = self._jdf.join(*jcols) + jcols = ListConverter().convert([_to_java_column(c) for c in cols[1:]], + self._sc._gateway._gateway_client) + jdf = self._jdf.sort(_to_java_column(cols[0]), + self._sc._jvm.Dsl.toColumns(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. """ + """ 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. """ - return self.head() + """ Return the first row. - def tail(self): - raise NotImplemented + >>> 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)] + """ if isinstance(item, basestring): - return Column(self._jdf.apply(item)) + jc = self._jdf.apply(item) + return Column(jc, self.sql_ctx) # TODO projection raise IndexError def __getattr__(self, name): - """ Return the column by given name """ + """ Return the column by given name + + >>> df.age.collect() + [Row(age=2), Row(age=5)] + """ if name.startswith("__"): raise AttributeError(name) - return Column(self._jdf.apply(name)) - - def alias(self, name): - """ Alias the current DataFrame """ - return DataFrame(getattr(self._jdf, "as")(name), self.sql_ctx) + jc = self._jdf.apply(name) + return Column(jc, self.sql_ctx) def select(self, *cols): - """ Selecting a set of expressions.:: - - df.select() - df.select('colA', 'colB') - df.select(df.colA, df.colB + 1) - + """ 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).As('age')).collect() + [Row(name=u'Alice', age=12), Row(name=u'Bob', age=15)] """ if not cols: cols = ["*"] - if isinstance(cols[0], basestring): - cols = [_create_column_from_name(n) for n in cols] - else: - cols = [c._jc for c in cols] - jcols = ListConverter().convert(cols, self._sc._gateway._gateway_client) + 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.Dsl.toColumns(jcols)) return DataFrame(jdf, self.sql_ctx) def filter(self, condition): - """ Filtering rows using the given condition:: - - df.filter(df.age > 15) - df.where(df.age > 15) + """ Filtering rows using the given condition. + >>> df.filter(df.age > 3).collect() + [Row(age=5, name=u'Bob')] + >>> df.where(df.age == 2).collect() + [Row(age=2, name=u'Alice')] """ return DataFrame(self._jdf.filter(condition._jc), self.sql_ctx) where = filter def groupBy(self, *cols): - """ Group the [[DataFrame]] using the specified columns, + """ Group the :class:`DataFrame` using the specified columns, so we can run aggregation on them. See :class:`GroupedDataFrame` - for all the available aggregate functions:: - - df.groupBy(df.department).avg() - df.groupBy("department", "gender").agg({ - "salary": "avg", - "age": "max", - }) + 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)] """ - if cols and isinstance(cols[0], basestring): - cols = [_create_column_from_name(n) for n in cols] - else: - cols = [c._jc for c in cols] - jcols = ListConverter().convert(cols, self._sc._gateway._gateway_client) + 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.Dsl.toColumns(jcols)) return GroupedDataFrame(jdf, self.sql_ctx) def agg(self, *exprs): - """ Aggregate on the entire [[DataFrame]] without groups - (shorthand for df.groupBy.agg()):: - - df.agg({"age": "max", "salary": "avg"}) + """ 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) @@ -2213,7 +2248,7 @@ def unionAll(self, other): return DataFrame(self._jdf.unionAll(other._jdf), self.sql_ctx) def intersect(self, other): - """ Return a new [[DataFrame]] containing rows only in + """ Return a new :class:`DataFrame` containing rows only in both this frame and another frame. This is equivalent to `INTERSECT` in SQL. @@ -2221,7 +2256,7 @@ def intersect(self, other): return DataFrame(self._jdf.intersect(other._jdf), self.sql_ctx) def subtract(self, other): - """ Return a new [[DataFrame]] containing rows in this frame + """ Return a new :class:`DataFrame` containing rows in this frame but not in another frame. This is equivalent to `EXCEPT` in SQL. @@ -2229,7 +2264,11 @@ def subtract(self, other): return DataFrame(getattr(self._jdf, "except")(other._jdf), self.sql_ctx) def sample(self, withReplacement, fraction, seed=None): - """ Return a new DataFrame by sampling a fraction of rows. """ + """ Return a new DataFrame by sampling a fraction of rows. + + >>> df.sample(False, 0.5, 10).collect() + [Row(age=2, name=u'Alice')] + """ if seed is None: jdf = self._jdf.sample(withReplacement, fraction) else: @@ -2237,11 +2276,12 @@ def sample(self, withReplacement, fraction, seed=None): return DataFrame(jdf, self.sql_ctx) def addColumn(self, colName, col): - """ Return a new [[DataFrame]] by adding a column. """ - return self.select('*', col.alias(colName)) + """ Return a new :class:`DataFrame` by adding a column. - def removeColumn(self, colName): - raise NotImplemented + >>> 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.As(colName)) # Having SchemaRDD for backward compatibility (for docs) @@ -2280,7 +2320,14 @@ def agg(self, *exprs): `sum`, `count`. :param exprs: list or aggregate columns or a map from column - name to agregate methods. + 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): @@ -2297,7 +2344,11 @@ def agg(self, *exprs): @dfapi def count(self): - """ Count the number of rows for each group. """ + """ 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): @@ -2349,18 +2400,25 @@ def sum(self): def _create_column_from_literal(literal): sc = SparkContext._active_spark_context - return sc._jvm.org.apache.spark.sql.Dsl.lit(literal) + return sc._jvm.Dsl.lit(literal) def _create_column_from_name(name): sc = SparkContext._active_spark_context - return sc._jvm.IncomputableColumn(name) + 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 _scalaMethod(name): """ Translate operators into methodName in Scala - For example: >>> _scalaMethod('+') '$plus' >>> _scalaMethod('>=') @@ -2371,37 +2429,34 @@ def _scalaMethod(name): return ''.join(SCALA_METHOD_MAPPINGS.get(c, c) for c in name) -def _unary_op(name): +def _unary_op(name, doc="unary operator"): """ Create a method for given unary operator """ def _(self): - return Column(getattr(self._jc, _scalaMethod(name))(), self._jdf, self.sql_ctx) + jc = getattr(self._jc, _scalaMethod(name))() + return Column(jc, self.sql_ctx) + _.__doc__ = doc return _ -def _bin_op(name, pass_literal_through=True): +def _bin_op(name, doc="binary operator"): """ Create a method for given binary operator - - Keyword arguments: - pass_literal_through -- whether to pass literal value directly through to the JVM. """ def _(self, other): - if isinstance(other, Column): - jc = other._jc - else: - if pass_literal_through: - jc = other - else: - jc = _create_column_from_literal(other) - return Column(getattr(self._jc, _scalaMethod(name))(jc), self._jdf, self.sql_ctx) + jc = other._jc if isinstance(other, Column) else other + njc = getattr(self._jc, _scalaMethod(name))(jc) + return Column(njc, self.sql_ctx) + _.__doc__ = doc return _ -def _reverse_op(name): +def _reverse_op(name, doc="binary operator"): """ Create a method for binary operator (this object is on right side) """ def _(self, other): - return Column(getattr(_create_column_from_literal(other), _scalaMethod(name))(self._jc), - self._jdf, self.sql_ctx) + jother = _create_column_from_literal(other) + jc = getattr(jother, _scalaMethod(name))(self._jc) + return Column(jc, self.sql_ctx) + _.__doc__ = doc return _ @@ -2410,20 +2465,20 @@ 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"] + `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 - }}} + # 2. Create from an expression + df.colName + 1 + 1 / df.colName """ - def __init__(self, jc, jdf=None, sql_ctx=None): + def __init__(self, jc, sql_ctx=None): self._jc = jc - super(Column, self).__init__(jdf, sql_ctx) + super(Column, self).__init__(jc, sql_ctx) # arithmetic operators __neg__ = _unary_op("unary_-") @@ -2438,8 +2493,6 @@ def __init__(self, jc, jdf=None, sql_ctx=None): __rdiv__ = _reverse_op("/") __rmod__ = _reverse_op("%") __abs__ = _unary_op("abs") - abs = _unary_op("abs") - sqrt = _unary_op("sqrt") # logistic operators __eq__ = _bin_op("===") @@ -2448,47 +2501,45 @@ def __init__(self, jc, jdf=None, sql_ctx=None): __le__ = _bin_op("<=") __ge__ = _bin_op(">=") __gt__ = _bin_op(">") - # `and`, `or`, `not` cannot be overloaded in Python - And = _bin_op('&&') - Or = _bin_op('||') - Not = _unary_op('unary_!') - - # bitwise operators - __and__ = _bin_op("&") - __or__ = _bin_op("|") - __invert__ = _unary_op("unary_~") - __xor__ = _bin_op("^") - # __lshift__ = _bin_op("<<") - # __rshift__ = _bin_op(">>") - __rand__ = _bin_op("&") - __ror__ = _bin_op("|") - __rxor__ = _bin_op("^") - # __rlshift__ = _reverse_op("<<") - # __rrshift__ = _reverse_op(">>") + + # `and`, `or`, `not` cannot be overloaded in Python, + # so use bitwise operators as boolean operators + __and__ = _bin_op('&&') + __or__ = _bin_op('||') + __invert__ = _unary_op('unary_!') + __rand__ = _bin_op("&&") + __ror__ = _bin_op("||") # container operators __contains__ = _bin_op("contains") __getitem__ = _bin_op("getItem") - # __getattr__ = _bin_op("getField") + 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") - upper = _unary_op("upper") - lower = _unary_op("lower") - def substr(self, startPos, pos): - if type(startPos) != type(pos): + 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, pos) + jc = self._jc.substr(startPos, length) elif isinstance(startPos, Column): - jc = self._jc.substr(startPos._jc, pos._jc) + jc = self._jc.substr(startPos._jc, length._jc) else: raise TypeError("Unexpected type: %s" % type(startPos)) - return Column(jc, self._jdf, self.sql_ctx) + return Column(jc, self.sql_ctx) __getslice__ = substr @@ -2496,55 +2547,89 @@ def substr(self, startPos, pos): asc = _unary_op("asc") desc = _unary_op("desc") - isNull = _unary_op("isNull") - isNotNull = _unary_op("isNotNull") + isNull = _unary_op("isNull", "True if the current expression is null.") + isNotNull = _unary_op("isNotNull", "True if the current expression is not null.") # `as` is keyword def alias(self, alias): - return Column(getattr(self._jsc, "as")(alias), self._jdf, self.sql_ctx) + """Return a alias for this column + + >>> df.age.As("age2").collect() + [Row(age2=2), Row(age2=5)] + >>> df.age.alias("age2").collect() + [Row(age2=2), Row(age2=5)] + """ + return Column(getattr(self._jc, "as")(alias), self.sql_ctx) + As = alias def cast(self, dataType): + """ Convert the column into type `dataType` + + >>> df.select(df.age.cast("string").As('ages')).collect() + [Row(ages=u'2'), Row(ages=u'5')] + >>> df.select(df.age.cast(StringType()).As('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 - jdt = ssql_ctx.parseDataType(dataType.json()) - return Column(self._jc.cast(jdt), self._jdf, self.sql_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_java_column(col): - if isinstance(col, Column): - jcol = col._jc - else: - jcol = _create_column_from_name(col) - return jcol - - -def _aggregate_func(name): +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 Aggregator(object): +class Dsl(object): """ A collections of builtin aggregators """ - AGGS = [ - 'lit', 'col', 'column', 'upper', 'lower', 'sqrt', 'abs', - 'min', 'max', 'first', 'last', 'count', 'avg', 'mean', 'sum', 'sumDistinct', - ] - for _name in AGGS: - locals()[_name] = _aggregate_func(_name) - del _name + 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).As('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) @@ -2554,6 +2639,12 @@ def countDistinct(col, *cols): @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).As('c')).collect() + [Row(c=2)] + """ sc = SparkContext._active_spark_context if rsd is None: jc = sc._jvm.Dsl.approxCountDistinct(_to_java_column(col)) @@ -2568,16 +2659,20 @@ def _test(): # let doctest run in pyspark.sql, so DataTypes can be picklable import pyspark.sql from pyspark.sql import Row, SQLContext - from pyspark.tests import ExamplePoint, ExamplePointUDT + from pyspark.sql_tests import ExamplePoint, ExamplePointUDT globs = pyspark.sql.__dict__.copy() sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc - globs['sqlCtx'] = SQLContext(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 = [ diff --git a/python/pyspark/sql_tests.py b/python/pyspark/sql_tests.py new file mode 100644 index 0000000000000..d314f46e8d2d5 --- /dev/null +++ b/python/pyspark/sql_tests.py @@ -0,0 +1,299 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Unit tests for pyspark.sql; additional tests are implemented as doctests in +individual modules. +""" +import os +import sys +import pydoc +import shutil +import tempfile + +if sys.version_info[:2] <= (2, 6): + try: + import unittest2 as unittest + except ImportError: + sys.stderr.write('Please install unittest2 to test with Python 2.6 or earlier') + sys.exit(1) +else: + import unittest + +from pyspark.sql import SQLContext, IntegerType, Row, ArrayType, StructType, StructField, \ + UserDefinedType, DoubleType +from pyspark.tests import ReusedPySparkTestCase + + +class ExamplePointUDT(UserDefinedType): + """ + User-defined type (UDT) for ExamplePoint. + """ + + @classmethod + def sqlType(self): + return ArrayType(DoubleType(), False) + + @classmethod + def module(cls): + return 'pyspark.tests' + + @classmethod + def scalaUDT(cls): + return 'org.apache.spark.sql.test.ExamplePointUDT' + + def serialize(self, obj): + return [obj.x, obj.y] + + def deserialize(self, datum): + return ExamplePoint(datum[0], datum[1]) + + +class ExamplePoint: + """ + An example class to demonstrate UDT in Scala, Java, and Python. + """ + + __UDT__ = ExamplePointUDT() + + def __init__(self, x, y): + self.x = x + self.y = y + + def __repr__(self): + return "ExamplePoint(%s,%s)" % (self.x, self.y) + + def __str__(self): + return "(%s,%s)" % (self.x, self.y) + + def __eq__(self, other): + return isinstance(other, ExamplePoint) and \ + other.x == self.x and other.y == self.y + + +class SQLTests(ReusedPySparkTestCase): + + @classmethod + def setUpClass(cls): + ReusedPySparkTestCase.setUpClass() + cls.tempdir = tempfile.NamedTemporaryFile(delete=False) + os.unlink(cls.tempdir.name) + 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) + + @classmethod + def tearDownClass(cls): + ReusedPySparkTestCase.tearDownClass() + shutil.rmtree(cls.tempdir.name, ignore_errors=True) + + def test_udf(self): + self.sqlCtx.registerFunction("twoArgs", lambda x, y: len(x) + y, IntegerType()) + [row] = self.sqlCtx.sql("SELECT twoArgs('test', 1)").collect() + self.assertEqual(row[0], 5) + + def test_udf2(self): + self.sqlCtx.registerFunction("strlen", lambda string: len(string), IntegerType()) + self.sqlCtx.inferSchema(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.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(1, l2) + + def test_broadcast_in_udf(self): + bar = {"a": "aa", "b": "bb", "c": "abc"} + foo = self.sc.broadcast(bar) + self.sqlCtx.registerFunction("MYUDF", lambda x: foo.value[x] if x else '') + [res] = self.sqlCtx.sql("SELECT MYUDF('c')").collect() + self.assertEqual("abc", res[0]) + [res] = self.sqlCtx.sql("SELECT MYUDF('')").collect() + self.assertEqual("", res[0]) + + def test_basic_functions(self): + rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}']) + df = self.sqlCtx.jsonRDD(rdd) + df.count() + df.collect() + df.schema() + + # cache and checkpoint + self.assertFalse(df.is_cached) + df.persist() + df.unpersist() + df.cache() + self.assertTrue(df.is_cached) + self.assertEqual(2, df.count()) + + df.registerTempTable("temp") + df = self.sqlCtx.sql("select foo from temp") + df.count() + df.collect() + + 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()) + 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()) + 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) + row = df.head() + self.assertEqual(1, len(row.l)) + self.assertEqual(1, row.l[0].a) + self.assertEqual("2", row.d["key"].d) + + l = df.map(lambda x: x.l).first() + self.assertEqual(1, len(l)) + self.assertEqual('s', l[0].b) + + d = df.map(lambda x: x.d).first() + self.assertEqual(1, len(d)) + self.assertEqual(1.0, d["key"].c) + + row = df.map(lambda x: x.d["key"]).first() + self.assertEqual(1.0, row.c) + self.assertEqual("2", row.d) + + 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) + 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) + 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") + result = self.sqlCtx.sql("SELECT l[0].a from test2 where d['key'].d = '2'") + self.assertEqual(1, result.head()[0]) + + def test_struct_in_map(self): + d = [Row(m={Row(i=1): Row(s="")})] + rdd = self.sc.parallelize(d) + df = self.sqlCtx.inferSchema(rdd) + k, v = df.head().m.items()[0] + self.assertEqual(1, k.i) + self.assertEqual("", v.s) + + 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.registerTempTable("test") + row = self.sqlCtx.sql("select l, d from test").head() + self.assertEqual(1, row.asDict()["l"][0].a) + self.assertEqual(1.0, row.asDict()['d']['key'].c) + + 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) + 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") + point = self.sqlCtx.sql("SELECT point FROM labeled_point").head().point + self.assertEqual(point, ExamplePoint(1.0, 2.0)) + + def test_apply_schema_with_udt(self): + 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), + StructField("point", ExamplePointUDT(), False)]) + df = self.sqlCtx.applySchema(rdd, 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.inferSchema(rdd) + output_dir = os.path.join(self.tempdir.name, "labeled_point") + df0.saveAsParquetFile(output_dir) + df1 = self.sqlCtx.parquetFile(output_dir) + point = df1.head().point + 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 + self.assertTrue(isinstance((- ci - 1 - 2) % 3 * 2.5 / 3.5, Column)) + rcc = (1 + ci), (1 - ci), (1 * ci), (1 / ci), (1 % ci) + self.assertTrue(all(isinstance(c, Column) for c in rcc)) + cb = [ci == 5, ci != 0, ci > 3, ci < 4, ci >= 0, ci <= 7, ci and cs, ci or cs] + self.assertTrue(all(isinstance(c, Column) for c in cb)) + cbool = (ci & ci), (ci | ci), (~ci) + self.assertTrue(all(isinstance(c, Column) for c in cbool)) + css = cs.like('a'), cs.rlike('a'), cs.asc(), cs.desc(), cs.startswith('a'), cs.endswith('a') + self.assertTrue(all(isinstance(c, Column) for c in css)) + self.assertTrue(isinstance(ci.cast(LongType()), Column)) + + def test_column_select(self): + df = self.df + self.assertEqual(self.testData, df.select("*").collect()) + self.assertEqual(self.testData, df.select(df.key, df.value).collect()) + self.assertEqual([Row(value='1')], df.where(df.key == 1).select(df.value).collect()) + + def test_aggregator(self): + df = self.df + g = df.groupBy() + 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]) + + def test_help_command(self): + # Regression test for SPARK-5464 + rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}']) + df = self.sqlCtx.jsonRDD(rdd) + # render_doc() reproduces the help() exception without printing output + pydoc.render_doc(df) + pydoc.render_doc(df.foo) + pydoc.render_doc(df.take(1)) + + +if __name__ == "__main__": + unittest.main() diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index c7d0622d65f25..b5e28c498040b 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -23,7 +23,6 @@ from fileinput import input from glob import glob import os -import pydoc import re import shutil import subprocess @@ -52,8 +51,6 @@ from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \ CloudPickleSerializer, CompressedSerializer, UTF8Deserializer, NoOpSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter -from pyspark.sql import SQLContext, IntegerType, Row, ArrayType, StructType, StructField, \ - UserDefinedType, DoubleType from pyspark import shuffle from pyspark.profiler import BasicProfiler @@ -795,264 +792,6 @@ def heavy_foo(x): rdd.foreach(heavy_foo) -class ExamplePointUDT(UserDefinedType): - """ - User-defined type (UDT) for ExamplePoint. - """ - - @classmethod - def sqlType(self): - return ArrayType(DoubleType(), False) - - @classmethod - def module(cls): - return 'pyspark.tests' - - @classmethod - def scalaUDT(cls): - return 'org.apache.spark.sql.test.ExamplePointUDT' - - def serialize(self, obj): - return [obj.x, obj.y] - - def deserialize(self, datum): - return ExamplePoint(datum[0], datum[1]) - - -class ExamplePoint: - """ - An example class to demonstrate UDT in Scala, Java, and Python. - """ - - __UDT__ = ExamplePointUDT() - - def __init__(self, x, y): - self.x = x - self.y = y - - def __repr__(self): - return "ExamplePoint(%s,%s)" % (self.x, self.y) - - def __str__(self): - return "(%s,%s)" % (self.x, self.y) - - def __eq__(self, other): - return isinstance(other, ExamplePoint) and \ - other.x == self.x and other.y == self.y - - -class SQLTests(ReusedPySparkTestCase): - - @classmethod - def setUpClass(cls): - ReusedPySparkTestCase.setUpClass() - cls.tempdir = tempfile.NamedTemporaryFile(delete=False) - os.unlink(cls.tempdir.name) - - @classmethod - def tearDownClass(cls): - ReusedPySparkTestCase.tearDownClass() - shutil.rmtree(cls.tempdir.name, ignore_errors=True) - - def setUp(self): - self.sqlCtx = SQLContext(self.sc) - self.testData = [Row(key=i, value=str(i)) for i in range(100)] - rdd = self.sc.parallelize(self.testData) - self.df = self.sqlCtx.inferSchema(rdd) - - def test_udf(self): - self.sqlCtx.registerFunction("twoArgs", lambda x, y: len(x) + y, IntegerType()) - [row] = self.sqlCtx.sql("SELECT twoArgs('test', 1)").collect() - self.assertEqual(row[0], 5) - - def test_udf2(self): - self.sqlCtx.registerFunction("strlen", lambda string: len(string), IntegerType()) - self.sqlCtx.inferSchema(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.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(1, l2) - - def test_broadcast_in_udf(self): - bar = {"a": "aa", "b": "bb", "c": "abc"} - foo = self.sc.broadcast(bar) - self.sqlCtx.registerFunction("MYUDF", lambda x: foo.value[x] if x else '') - [res] = self.sqlCtx.sql("SELECT MYUDF('c')").collect() - self.assertEqual("abc", res[0]) - [res] = self.sqlCtx.sql("SELECT MYUDF('')").collect() - self.assertEqual("", res[0]) - - def test_basic_functions(self): - rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}']) - df = self.sqlCtx.jsonRDD(rdd) - df.count() - df.collect() - df.schema() - - # cache and checkpoint - self.assertFalse(df.is_cached) - df.persist() - df.unpersist() - df.cache() - self.assertTrue(df.is_cached) - self.assertEqual(2, df.count()) - - df.registerTempTable("temp") - df = self.sqlCtx.sql("select foo from temp") - df.count() - df.collect() - - 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()) - 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()) - 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) - row = df.head() - self.assertEqual(1, len(row.l)) - self.assertEqual(1, row.l[0].a) - self.assertEqual("2", row.d["key"].d) - - l = df.map(lambda x: x.l).first() - self.assertEqual(1, len(l)) - self.assertEqual('s', l[0].b) - - d = df.map(lambda x: x.d).first() - self.assertEqual(1, len(d)) - self.assertEqual(1.0, d["key"].c) - - row = df.map(lambda x: x.d["key"]).first() - self.assertEqual(1.0, row.c) - self.assertEqual("2", row.d) - - 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) - 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) - 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") - result = self.sqlCtx.sql("SELECT l[0].a from test2 where d['key'].d = '2'") - self.assertEqual(1, result.head()[0]) - - def test_struct_in_map(self): - d = [Row(m={Row(i=1): Row(s="")})] - rdd = self.sc.parallelize(d) - df = self.sqlCtx.inferSchema(rdd) - k, v = df.head().m.items()[0] - self.assertEqual(1, k.i) - self.assertEqual("", v.s) - - 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.registerTempTable("test") - row = self.sqlCtx.sql("select l, d from test").head() - self.assertEqual(1, row.asDict()["l"][0].a) - self.assertEqual(1.0, row.asDict()['d']['key'].c) - - def test_infer_schema_with_udt(self): - from pyspark.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) - 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") - point = self.sqlCtx.sql("SELECT point FROM labeled_point").head().point - self.assertEqual(point, ExamplePoint(1.0, 2.0)) - - def test_apply_schema_with_udt(self): - from pyspark.tests import ExamplePoint, ExamplePointUDT - row = (1.0, ExamplePoint(1.0, 2.0)) - rdd = self.sc.parallelize([row]) - schema = StructType([StructField("label", DoubleType(), False), - StructField("point", ExamplePointUDT(), False)]) - df = self.sqlCtx.applySchema(rdd, schema) - point = df.head().point - self.assertEquals(point, ExamplePoint(1.0, 2.0)) - - def test_parquet_with_udt(self): - from pyspark.tests import ExamplePoint - row = Row(label=1.0, point=ExamplePoint(1.0, 2.0)) - rdd = self.sc.parallelize([row]) - df0 = self.sqlCtx.inferSchema(rdd) - output_dir = os.path.join(self.tempdir.name, "labeled_point") - df0.saveAsParquetFile(output_dir) - df1 = self.sqlCtx.parquetFile(output_dir) - point = df1.head().point - 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 - self.assertTrue(isinstance((- ci - 1 - 2) % 3 * 2.5 / 3.5, Column)) - rcc = (1 + ci), (1 - ci), (1 * ci), (1 / ci), (1 % ci) - self.assertTrue(all(isinstance(c, Column) for c in rcc)) - cb = [ci == 5, ci != 0, ci > 3, ci < 4, ci >= 0, ci <= 7, ci and cs, ci or cs] - self.assertTrue(all(isinstance(c, Column) for c in cb)) - cbit = (ci & ci), (ci | ci), (ci ^ ci), (~ci) - self.assertTrue(all(isinstance(c, Column) for c in cbit)) - css = cs.like('a'), cs.rlike('a'), cs.asc(), cs.desc(), cs.startswith('a'), cs.endswith('a') - self.assertTrue(all(isinstance(c, Column) for c in css)) - self.assertTrue(isinstance(ci.cast(LongType()), Column)) - - def test_column_select(self): - df = self.df - self.assertEqual(self.testData, df.select("*").collect()) - self.assertEqual(self.testData, df.select(df.key, df.value).collect()) - self.assertEqual([Row(value='1')], df.where(df.key == 1).select(df.value).collect()) - - def test_aggregator(self): - df = self.df - g = df.groupBy() - 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 Aggregator as Agg - self.assertEqual((0, u'99'), tuple(g.agg(Agg.first(df.key), Agg.last(df.value)).first())) - self.assertTrue(95 < g.agg(Agg.approxCountDistinct(df.key)).first()[0]) - self.assertEqual(100, g.agg(Agg.countDistinct(df.value)).first()[0]) - - def test_help_command(self): - # Regression test for SPARK-5464 - rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}']) - df = self.sqlCtx.jsonRDD(rdd) - # render_doc() reproduces the help() exception without printing output - pydoc.render_doc(df) - pydoc.render_doc(df.foo) - pydoc.render_doc(df.take(1)) - - class InputFormatTests(ReusedPySparkTestCase): @classmethod diff --git a/python/run-tests b/python/run-tests index e91f1a875d356..649a2c44d187b 100755 --- a/python/run-tests +++ b/python/run-tests @@ -65,6 +65,7 @@ function run_core_tests() { function run_sql_tests() { echo "Run sql tests ..." run_test "pyspark/sql.py" + run_test "pyspark/sql_tests.py" } function run_mllib_tests() { 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 4aa37219e13a6..ddce77deb83e1 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 @@ -115,15 +115,6 @@ trait Column extends DataFrame { */ def unary_- : Column = exprToColumn(UnaryMinus(expr)) - /** - * Bitwise NOT. - * {{{ - * // Scala: select the flags column and negate every bit. - * df.select( ~df("flags") ) - * }}} - */ - def unary_~ : Column = exprToColumn(BitwiseNot(expr)) - /** * Inversion of boolean expression, i.e. NOT. * {{ @@ -362,27 +353,6 @@ trait Column extends DataFrame { */ def and(other: Column): Column = this && other - /** - * Bitwise AND. - */ - def & (other: Any): Column = constructColumn(other) { o => - BitwiseAnd(expr, o.expr) - } - - /** - * Bitwise OR with an expression. - */ - def | (other: Any): Column = constructColumn(other) { o => - BitwiseOr(expr, o.expr) - } - - /** - * Bitwise XOR with an expression. - */ - def ^ (other: Any): Column = constructColumn(other) { o => - BitwiseXor(expr, o.expr) - } - /** * Sum of this expression and another expression. * {{{ @@ -527,16 +497,16 @@ trait Column extends DataFrame { * @param startPos expression for the starting position. * @param len expression for the length of the substring. */ - def substr(startPos: Column, len: Column): Column = { - new IncomputableColumn(Substring(expr, startPos.expr, len.expr)) - } + def substr(startPos: Column, len: Column): Column = + exprToColumn(Substring(expr, startPos.expr, len.expr), computable = false) /** * An expression that returns a substring. * @param startPos starting position. * @param len length of the substring. */ - def substr(startPos: Int, len: Int): Column = this.substr(lit(startPos), lit(len)) + def substr(startPos: Int, len: Int): Column = + exprToColumn(Substring(expr, lit(startPos).expr, lit(len).expr)) def contains(other: Any): Column = constructColumn(other) { o => Contains(expr, o.expr) 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 006b16fbe07bd..e6f622e87f7a4 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.tests.ExamplePointUDT" + override def pyUDT: String = "pyspark.sql_tests.ExamplePointUDT" override def serialize(obj: Any): Seq[Double] = { obj match { From e380d2d46c92b319eafe30974ac7c1509081fca4 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Tue, 3 Feb 2015 17:02:42 -0800 Subject: [PATCH 052/578] [SPARK-5520][MLlib] Make FP-Growth implementation take generic item types (WIP) Make FPGrowth.run API take generic item types: `def run[Item: ClassTag, Basket <: Iterable[Item]](data: RDD[Basket]): FPGrowthModel[Item]` so that user can invoke it by run[String, Seq[String]], run[Int, Seq[Int]], run[Int, List[Int]], etc. Scala part is done, while java part is still in progress Author: Jacky Li Author: Jacky Li Author: Xiangrui Meng Closes #4340 from jackylk/SPARK-5520-WIP and squashes the following commits: f5acf84 [Jacky Li] Merge pull request #2 from mengxr/SPARK-5520 63073d0 [Xiangrui Meng] update to make generic FPGrowth Java-friendly 737d8bb [Jacky Li] fix scalastyle 793f85c [Jacky Li] add Java test case 7783351 [Jacky Li] add generic support in FPGrowth --- .../org/apache/spark/mllib/fpm/FPGrowth.scala | 50 +++++++---- .../spark/mllib/fpm/JavaFPGrowthSuite.java | 84 +++++++++++++++++++ .../spark/mllib/fpm/FPGrowthSuite.scala | 51 ++++++++++- 3 files changed, 170 insertions(+), 15 deletions(-) create mode 100644 mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java 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 9591c7966e06a..1433ee9a0dd5a 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 @@ -18,14 +18,31 @@ package org.apache.spark.mllib.fpm import java.{util => ju} +import java.lang.{Iterable => JavaIterable} import scala.collection.mutable +import scala.collection.JavaConverters._ +import scala.reflect.ClassTag -import org.apache.spark.{SparkException, HashPartitioner, Logging, Partitioner} +import org.apache.spark.{HashPartitioner, Logging, Partitioner, SparkException} +import org.apache.spark.api.java.{JavaPairRDD, JavaRDD} +import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -class FPGrowthModel(val freqItemsets: RDD[(Array[String], Long)]) extends Serializable +/** + * Model trained by [[FPGrowth]], which holds frequent itemsets. + * @param freqItemsets frequent itemset, which is an RDD of (itemset, frequency) pairs + * @tparam Item item type + */ +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]] + } +} /** * This class implements Parallel FP-growth algorithm to do frequent pattern matching on input data. @@ -69,7 +86,7 @@ class FPGrowth private ( * @param data input data set, each element contains a transaction * @return an [[FPGrowthModel]] */ - def run(data: RDD[Array[String]]): FPGrowthModel = { + def run[Item: ClassTag](data: RDD[Array[Item]]): FPGrowthModel[Item] = { if (data.getStorageLevel == StorageLevel.NONE) { logWarning("Input data is not cached.") } @@ -82,19 +99,24 @@ class FPGrowth private ( new FPGrowthModel(freqItemsets) } + def run[Item, Basket <: JavaIterable[Item]](data: JavaRDD[Basket]): FPGrowthModel[Item] = { + implicit val tag = fakeClassTag[Item] + run(data.rdd.map(_.asScala.toArray)) + } + /** * Generates frequent items by filtering the input data using minimal support level. * @param minCount minimum count for frequent itemsets * @param partitioner partitioner used to distribute items * @return array of frequent pattern ordered by their frequencies */ - private def genFreqItems( - data: RDD[Array[String]], + private def genFreqItems[Item: ClassTag]( + data: RDD[Array[Item]], minCount: Long, - partitioner: Partitioner): Array[String] = { + partitioner: Partitioner): Array[Item] = { data.flatMap { t => val uniq = t.toSet - if (t.length != uniq.size) { + if (t.size != uniq.size) { throw new SparkException(s"Items in a transaction must be unique but got ${t.toSeq}.") } t @@ -114,11 +136,11 @@ class FPGrowth private ( * @param partitioner partitioner used to distribute transactions * @return an RDD of (frequent itemset, count) */ - private def genFreqItemsets( - data: RDD[Array[String]], + private def genFreqItemsets[Item: ClassTag]( + data: RDD[Array[Item]], minCount: Long, - freqItems: Array[String], - partitioner: Partitioner): RDD[(Array[String], Long)] = { + freqItems: Array[Item], + partitioner: Partitioner): RDD[(Array[Item], Long)] = { val itemToRank = freqItems.zipWithIndex.toMap data.flatMap { transaction => genCondTransactions(transaction, itemToRank, partitioner) @@ -139,9 +161,9 @@ class FPGrowth private ( * @param partitioner partitioner used to distribute transactions * @return a map of (target partition, conditional transaction) */ - private def genCondTransactions( - transaction: Array[String], - itemToRank: Map[String, Int], + private def genCondTransactions[Item: ClassTag]( + transaction: Array[Item], + itemToRank: Map[Item, Int], partitioner: Partitioner): mutable.Map[Int, Array[Int]] = { val output = mutable.Map.empty[Int, Array[Int]] // Filter the basket by frequent items pattern and sort their ranks. 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 new file mode 100644 index 0000000000000..851707c8a19c4 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.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 org.apache.spark.mllib.fpm; + +import java.io.Serializable; +import java.util.ArrayList; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import com.google.common.collect.Lists; +import static org.junit.Assert.*; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +public class JavaFPGrowthSuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaFPGrowth"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + public void runFPGrowth() { + + @SuppressWarnings("unchecked") + JavaRDD> rdd = 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(); + + FPGrowthModel model6 = fpg + .setMinSupport(0.9) + .setNumPartitions(1) + .run(rdd); + assertEquals(0, model6.javaFreqItemsets().count()); + + FPGrowthModel model3 = fpg + .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()); + + FPGrowthModel model1 = fpg + .setMinSupport(0.1) + .setNumPartitions(8) + .run(rdd); + assertEquals(625, model1.javaFreqItemsets().count()); + } +} 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 71ef60da6dd32..68128284b8608 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 @@ -22,7 +22,8 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext class FPGrowthSuite extends FunSuite with MLlibTestSparkContext { - test("FP-Growth") { + + test("FP-Growth using String type") { val transactions = Seq( "r z h k p", "z y x w v u t s", @@ -70,4 +71,52 @@ class FPGrowthSuite extends FunSuite with MLlibTestSparkContext { .run(rdd) assert(model1.freqItemsets.count() === 625) } + + test("FP-Growth using Int type") { + val transactions = Seq( + "1 2 3", + "1 2 3 4", + "5 4 3 2 1", + "6 5 4 3 2 1", + "2 4", + "1 3", + "1 7") + .map(_.split(" ").map(_.toInt).toArray) + val rdd = sc.parallelize(transactions, 2).cache() + + val fpg = new FPGrowth() + + val model6 = fpg + .setMinSupport(0.9) + .setNumPartitions(1) + .run(rdd) + assert(model6.freqItemsets.count() === 0) + + val model3 = fpg + .setMinSupport(0.5) + .setNumPartitions(2) + .run(rdd) + assert(model3.freqItemsets.first()._1.getClass === Array(1).getClass, + "frequent itemsets should use primitive arrays") + val freqItemsets3 = model3.freqItemsets.collect().map { case (items, count) => + (items.toSet, count) + } + val expected = Set( + (Set(1), 6L), (Set(2), 5L), (Set(3), 5L), (Set(4), 4L), + (Set(1, 2), 4L), (Set(1, 3), 5L), (Set(2, 3), 4L), + (Set(2, 4), 4L), (Set(1, 2, 3), 4L)) + assert(freqItemsets3.toSet === expected) + + val model2 = fpg + .setMinSupport(0.3) + .setNumPartitions(4) + .run(rdd) + assert(model2.freqItemsets.count() === 15) + + val model1 = fpg + .setMinSupport(0.1) + .setNumPartitions(8) + .run(rdd) + assert(model1.freqItemsets.count() === 65) + } } From 1077f2e1def6266aee6ad6f0640a8f46cd273e21 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 3 Feb 2015 20:07:46 -0800 Subject: [PATCH 053/578] [SPARK-5578][SQL][DataFrame] Provide a convenient way for Scala users to use UDFs A more convenient way to define user-defined functions. Author: Reynold Xin Closes #4345 from rxin/defineUDF and squashes the following commits: 639c0f8 [Reynold Xin] udf tests. 0a0b339 [Reynold Xin] defineUDF -> udf. b452b8d [Reynold Xin] Fix UDF registration. d2e42c3 [Reynold Xin] SQLContext.udf.register() returns a UserDefinedFunction also. 4333605 [Reynold Xin] [SQL][DataFrame] defineUDF. --- .../classification/LogisticRegression.scala | 10 +- .../spark/ml/feature/StandardScaler.scala | 6 +- .../apache/spark/ml/recommendation/ALS.scala | 14 +- .../org/apache/spark/sql/DataFrame.scala | 4 +- .../org/apache/spark/sql/DataFrameImpl.scala | 2 +- .../main/scala/org/apache/spark/sql/Dsl.scala | 194 ++++---- ...gistration.scala => UDFRegistration.scala} | 441 +++++------------- .../spark/sql/UserDefinedFunction.scala | 39 ++ .../org/apache/spark/sql/DataFrameSuite.scala | 6 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 6 +- .../spark/sql/UserDefinedTypeSuite.scala | 5 +- 11 files changed, 279 insertions(+), 448 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/{UdfRegistration.scala => UDFRegistration.scala} (56%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/UserDefinedFunction.scala 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 18be35ad59452..df90078de148f 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 @@ -132,14 +132,14 @@ class LogisticRegressionModel private[ml] ( override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { transformSchema(dataset.schema, paramMap, logging = true) val map = this.paramMap ++ paramMap - val scoreFunction: Vector => Double = (v) => { + val scoreFunction = udf((v: Vector) => { val margin = BLAS.dot(v, weights) 1.0 / (1.0 + math.exp(-margin)) - } + } : Double) val t = map(threshold) - val predictFunction: Double => Double = (score) => { if (score > t) 1.0 else 0.0 } + val predictFunction = udf((score: Double) => { if (score > t) 1.0 else 0.0 } : Double) dataset - .select($"*", callUDF(scoreFunction, col(map(featuresCol))).as(map(scoreCol))) - .select($"*", callUDF(predictFunction, col(map(scoreCol))).as(map(predictionCol))) + .select($"*", scoreFunction(col(map(featuresCol))).as(map(scoreCol))) + .select($"*", predictFunction(col(map(scoreCol))).as(map(predictionCol))) } } 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 01a4f5eb205e5..4745a7ae95679 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 @@ -81,10 +81,8 @@ class StandardScalerModel private[ml] ( override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { transformSchema(dataset.schema, paramMap, logging = true) val map = this.paramMap ++ paramMap - val scale: (Vector) => Vector = (v) => { - scaler.transform(v) - } - dataset.select($"*", callUDF(scale, col(map(inputCol))).as(map(outputCol))) + val scale = udf((v: Vector) => { scaler.transform(v) } : Vector) + dataset.select($"*", scale(col(map(inputCol))).as(map(outputCol))) } private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { 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 511cb2fe4005e..c7bec7a845a11 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 @@ -126,22 +126,20 @@ class ALSModel private[ml] ( val map = this.paramMap ++ paramMap val users = userFactors.toDataFrame("id", "features") val items = itemFactors.toDataFrame("id", "features") - val predict: (Seq[Float], Seq[Float]) => Float = (userFeatures, itemFeatures) => { + + // Register a UDF for DataFrame, and then + // create a new column named map(predictionCol) by running the predict UDF. + val predict = udf((userFeatures: Seq[Float], itemFeatures: Seq[Float]) => { if (userFeatures != null && itemFeatures != null) { blas.sdot(k, userFeatures.toArray, 1, itemFeatures.toArray, 1) } else { Float.NaN } - } - val inputColumns = dataset.schema.fieldNames - val prediction = callUDF(predict, users("features"), items("features")).as(map(predictionCol)) - val outputColumns = inputColumns.map(f => dataset(f)) :+ prediction + } : Float) dataset .join(users, dataset(map(userCol)) === users("id"), "left") .join(items, dataset(map(itemCol)) === items("id"), "left") - .select(outputColumns: _*) - // TODO: Just use a dataset("*") - // .select(dataset("*"), prediction) + .select(dataset("*"), predict(users("features"), items("features")).as(map(predictionCol))) } override private[ml] def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { 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 f3bc07ae5238c..732b6855587dd 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 @@ -54,10 +54,10 @@ private[sql] object DataFrame { * }}} * * Note that the [[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 // in Scala - * }} + * }}} * * A more concrete 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 0b0623dc1fe75..a52bfa59a1365 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 @@ -329,7 +329,7 @@ private[sql] class DataFrameImpl protected[sql]( override def save(path: String): Unit = { val dataSourceName = sqlContext.conf.defaultDataSourceName - save(dataSourceName, ("path" -> path)) + save(dataSourceName, "path" -> path) } override def save( 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 71365c776d559..8cf59f0a1f099 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 @@ -186,15 +186,13 @@ object Dsl { (0 to 22).map { x => val types = (1 to x).foldRight("RT")((i, s) => {s"A$i, $s"}) val typeTags = (1 to x).map(i => s"A$i: TypeTag").foldLeft("RT: TypeTag")(_ + ", " + _) - val args = (1 to x).map(i => s"arg$i: Column").mkString(", ") - val argsInUdf = (1 to x).map(i => s"arg$i.expr").mkString(", ") println(s""" /** - * Call a Scala function of ${x} arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * 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. */ - def callUDF[$typeTags](f: Function$x[$types]${if (args.length > 0) ", " + args else ""}): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq($argsInUdf)) + def udf[$typeTags](f: Function$x[$types]): UserDefinedFunction = { + UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) }""") } @@ -214,187 +212,187 @@ object Dsl { } */ /** - * Call a Scala function of 0 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * 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. */ - def callUDF[RT: TypeTag](f: Function0[RT]): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq()) + def udf[RT: TypeTag](f: Function0[RT]): UserDefinedFunction = { + UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) } /** - * Call a Scala function of 1 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * 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. */ - def callUDF[RT: TypeTag, A1: TypeTag](f: Function1[A1, RT], arg1: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr)) + def udf[RT: TypeTag, A1: TypeTag](f: Function1[A1, RT]): UserDefinedFunction = { + UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) } /** - * Call a Scala function of 2 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * 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. */ - def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag](f: Function2[A1, A2, RT], arg1: Column, arg2: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr)) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag](f: Function2[A1, A2, RT]): UserDefinedFunction = { + UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) } /** - * Call a Scala function of 3 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * 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. */ - def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](f: Function3[A1, A2, A3, RT], arg1: Column, arg2: Column, arg3: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr)) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](f: Function3[A1, A2, A3, RT]): UserDefinedFunction = { + UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) } /** - * Call a Scala function of 4 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * 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. */ - def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](f: Function4[A1, A2, A3, A4, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr)) + 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) } /** - * Call a Scala function of 5 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * 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. */ - def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](f: Function5[A1, A2, A3, A4, A5, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr)) + 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) } /** - * Call a Scala function of 6 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * 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. */ - def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](f: Function6[A1, A2, A3, A4, A5, A6, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr)) + 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) } /** - * Call a Scala function of 7 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * 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. */ - def callUDF[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], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr)) + 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) } /** - * Call a Scala function of 8 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * 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. */ - def callUDF[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], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr)) + 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) } /** - * Call a Scala function of 9 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * 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. */ - def callUDF[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], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr)) + 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) } /** - * Call a Scala function of 10 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * 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. */ - def callUDF[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], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr)) + 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) } /** - * Call a Scala function of 11 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * Defines a user-defined function of 11 arguments as user-defined function (UDF). + * The data types are automatically inferred based on the function's signature. */ - def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag](f: Function11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr)) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag](f: Function11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, RT]): UserDefinedFunction = { + UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) } /** - * Call a Scala function of 12 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * Defines a user-defined function of 12 arguments as user-defined function (UDF). + * The data types are automatically inferred based on the function's signature. */ - def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag](f: Function12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr)) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag](f: Function12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, RT]): UserDefinedFunction = { + UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) } /** - * Call a Scala function of 13 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * Defines a user-defined function of 13 arguments as user-defined function (UDF). + * The data types are automatically inferred based on the function's signature. */ - def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag](f: Function13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr)) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag](f: Function13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, RT]): UserDefinedFunction = { + UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) } /** - * Call a Scala function of 14 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * Defines a user-defined function of 14 arguments as user-defined function (UDF). + * The data types are automatically inferred based on the function's signature. */ - def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag](f: Function14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr)) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag](f: Function14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, RT]): UserDefinedFunction = { + UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) } /** - * Call a Scala function of 15 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * Defines a user-defined function of 15 arguments as user-defined function (UDF). + * The data types are automatically inferred based on the function's signature. */ - def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag](f: Function15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr)) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag](f: Function15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, RT]): UserDefinedFunction = { + UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) } /** - * Call a Scala function of 16 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * Defines a user-defined function of 16 arguments as user-defined function (UDF). + * The data types are automatically inferred based on the function's signature. */ - def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag](f: Function16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr)) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag](f: Function16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, RT]): UserDefinedFunction = { + UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) } /** - * Call a Scala function of 17 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * Defines a user-defined function of 17 arguments as user-defined function (UDF). + * The data types are automatically inferred based on the function's signature. */ - def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag](f: Function17[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr)) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag](f: Function17[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, RT]): UserDefinedFunction = { + UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) } /** - * Call a Scala function of 18 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * Defines a user-defined function of 18 arguments as user-defined function (UDF). + * The data types are automatically inferred based on the function's signature. */ - def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag](f: Function18[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr)) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag](f: Function18[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, RT]): UserDefinedFunction = { + UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) } /** - * Call a Scala function of 19 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * Defines a user-defined function of 19 arguments as user-defined function (UDF). + * The data types are automatically inferred based on the function's signature. */ - def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag](f: Function19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr)) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag](f: Function19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, RT]): UserDefinedFunction = { + UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) } /** - * Call a Scala function of 20 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * Defines a user-defined function of 20 arguments as user-defined function (UDF). + * The data types are automatically inferred based on the function's signature. */ - def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag](f: Function20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column, arg20: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr, arg20.expr)) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag](f: Function20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, RT]): UserDefinedFunction = { + UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) } /** - * Call a Scala function of 21 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * Defines a user-defined function of 21 arguments as user-defined function (UDF). + * The data types are automatically inferred based on the function's signature. */ - def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag](f: Function21[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column, arg20: Column, arg21: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr, arg20.expr, arg21.expr)) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag](f: Function21[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, RT]): UserDefinedFunction = { + UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) } /** - * Call a Scala function of 22 arguments as user-defined function (UDF), and automatically - * infer the data types based on the function's signature. + * Defines a user-defined function of 22 arguments as user-defined function (UDF). + * The data types are automatically inferred based on the function's signature. */ - def callUDF[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag, A22: TypeTag](f: Function22[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, RT], arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column, arg20: Column, arg21: Column, arg22: Column): Column = { - ScalaUdf(f, ScalaReflection.schemaFor(typeTag[RT]).dataType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr, arg20.expr, arg21.expr, arg22.expr)) + def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag, A22: TypeTag](f: Function22[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, RT]): UserDefinedFunction = { + UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) } ////////////////////////////////////////////////////////////////////////////////////////////////// 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 similarity index 56% rename from sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala rename to sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index 1beb19437a8da..d8b0a3b26dbab 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 @@ -78,20 +78,21 @@ class UDFRegistration(sqlContext: SQLContext) extends Logging { // scalastyle:off - /* registerFunction 0-22 were generated by this script + /* register 0-22 were generated by this script (0 to 22).map { x => val types = (1 to x).foldRight("RT")((i, s) => {s"A$i, $s"}) val typeTags = (1 to x).map(i => s"A${i}: TypeTag").foldLeft("RT: TypeTag")(_ + ", " + _) - val argDocs = (1 to x).map(i => s" * @tparam A$i type of the UDF argument at position $i.").foldLeft("")(_ + "\n" + _) println(s""" /** * Register a Scala closure of ${x} arguments as user-defined function (UDF). - * @tparam RT return type of UDF.$argDocs + * @tparam RT return type of UDF. */ - def register[$typeTags](name: String, func: Function$x[$types]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + def register[$typeTags](name: String, func: Function$x[$types]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) }""") } @@ -116,462 +117,258 @@ class UDFRegistration(sqlContext: SQLContext) extends Logging { * Register a Scala closure of 0 arguments as user-defined function (UDF). * @tparam RT return type of UDF. */ - def register[RT: TypeTag](name: String, func: Function0[RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + def register[RT: TypeTag](name: String, func: Function0[RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 1 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. */ - def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 2 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 3 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 4 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 5 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. - * @tparam A5 type of the UDF argument at position 5. - */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 6 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. - * @tparam A5 type of the UDF argument at position 5. - * @tparam A6 type of the UDF argument at position 6. - */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](name: String, func: Function6[A1, A2, A3, A4, A5, A6, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](name: String, func: Function6[A1, A2, A3, A4, A5, A6, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 7 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. - * @tparam A5 type of the UDF argument at position 5. - * @tparam A6 type of the UDF argument at position 6. - * @tparam A7 type of the UDF argument at position 7. - */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](name: String, func: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](name: String, func: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 8 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. - * @tparam A5 type of the UDF argument at position 5. - * @tparam A6 type of the UDF argument at position 6. - * @tparam A7 type of the UDF argument at position 7. - * @tparam A8 type of the UDF argument at position 8. - */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](name: String, func: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](name: String, func: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 9 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. - * @tparam A5 type of the UDF argument at position 5. - * @tparam A6 type of the UDF argument at position 6. - * @tparam A7 type of the UDF argument at position 7. - * @tparam A8 type of the UDF argument at position 8. - * @tparam A9 type of the UDF argument at position 9. - */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](name: String, func: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](name: String, func: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 10 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. - * @tparam A5 type of the UDF argument at position 5. - * @tparam A6 type of the UDF argument at position 6. - * @tparam A7 type of the UDF argument at position 7. - * @tparam A8 type of the UDF argument at position 8. - * @tparam A9 type of the UDF argument at position 9. - * @tparam A10 type of the UDF argument at position 10. - */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](name: String, func: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](name: String, func: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 11 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. - * @tparam A5 type of the UDF argument at position 5. - * @tparam A6 type of the UDF argument at position 6. - * @tparam A7 type of the UDF argument at position 7. - * @tparam A8 type of the UDF argument at position 8. - * @tparam A9 type of the UDF argument at position 9. - * @tparam A10 type of the UDF argument at position 10. - * @tparam A11 type of the UDF argument at position 11. - */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag](name: String, func: Function11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag](name: String, func: Function11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 12 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. - * @tparam A5 type of the UDF argument at position 5. - * @tparam A6 type of the UDF argument at position 6. - * @tparam A7 type of the UDF argument at position 7. - * @tparam A8 type of the UDF argument at position 8. - * @tparam A9 type of the UDF argument at position 9. - * @tparam A10 type of the UDF argument at position 10. - * @tparam A11 type of the UDF argument at position 11. - * @tparam A12 type of the UDF argument at position 12. - */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag](name: String, func: Function12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag](name: String, func: Function12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 13 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. - * @tparam A5 type of the UDF argument at position 5. - * @tparam A6 type of the UDF argument at position 6. - * @tparam A7 type of the UDF argument at position 7. - * @tparam A8 type of the UDF argument at position 8. - * @tparam A9 type of the UDF argument at position 9. - * @tparam A10 type of the UDF argument at position 10. - * @tparam A11 type of the UDF argument at position 11. - * @tparam A12 type of the UDF argument at position 12. - * @tparam A13 type of the UDF argument at position 13. - */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag](name: String, func: Function13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag](name: String, func: Function13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 14 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. - * @tparam A5 type of the UDF argument at position 5. - * @tparam A6 type of the UDF argument at position 6. - * @tparam A7 type of the UDF argument at position 7. - * @tparam A8 type of the UDF argument at position 8. - * @tparam A9 type of the UDF argument at position 9. - * @tparam A10 type of the UDF argument at position 10. - * @tparam A11 type of the UDF argument at position 11. - * @tparam A12 type of the UDF argument at position 12. - * @tparam A13 type of the UDF argument at position 13. - * @tparam A14 type of the UDF argument at position 14. - */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag](name: String, func: Function14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag](name: String, func: Function14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 15 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. - * @tparam A5 type of the UDF argument at position 5. - * @tparam A6 type of the UDF argument at position 6. - * @tparam A7 type of the UDF argument at position 7. - * @tparam A8 type of the UDF argument at position 8. - * @tparam A9 type of the UDF argument at position 9. - * @tparam A10 type of the UDF argument at position 10. - * @tparam A11 type of the UDF argument at position 11. - * @tparam A12 type of the UDF argument at position 12. - * @tparam A13 type of the UDF argument at position 13. - * @tparam A14 type of the UDF argument at position 14. - * @tparam A15 type of the UDF argument at position 15. - */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag](name: String, func: Function15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag](name: String, func: Function15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 16 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. - * @tparam A5 type of the UDF argument at position 5. - * @tparam A6 type of the UDF argument at position 6. - * @tparam A7 type of the UDF argument at position 7. - * @tparam A8 type of the UDF argument at position 8. - * @tparam A9 type of the UDF argument at position 9. - * @tparam A10 type of the UDF argument at position 10. - * @tparam A11 type of the UDF argument at position 11. - * @tparam A12 type of the UDF argument at position 12. - * @tparam A13 type of the UDF argument at position 13. - * @tparam A14 type of the UDF argument at position 14. - * @tparam A15 type of the UDF argument at position 15. - * @tparam A16 type of the UDF argument at position 16. - */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag](name: String, func: Function16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag](name: String, func: Function16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 17 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. - * @tparam A5 type of the UDF argument at position 5. - * @tparam A6 type of the UDF argument at position 6. - * @tparam A7 type of the UDF argument at position 7. - * @tparam A8 type of the UDF argument at position 8. - * @tparam A9 type of the UDF argument at position 9. - * @tparam A10 type of the UDF argument at position 10. - * @tparam A11 type of the UDF argument at position 11. - * @tparam A12 type of the UDF argument at position 12. - * @tparam A13 type of the UDF argument at position 13. - * @tparam A14 type of the UDF argument at position 14. - * @tparam A15 type of the UDF argument at position 15. - * @tparam A16 type of the UDF argument at position 16. - * @tparam A17 type of the UDF argument at position 17. - */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag](name: String, func: Function17[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag](name: String, func: Function17[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 18 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. - * @tparam A5 type of the UDF argument at position 5. - * @tparam A6 type of the UDF argument at position 6. - * @tparam A7 type of the UDF argument at position 7. - * @tparam A8 type of the UDF argument at position 8. - * @tparam A9 type of the UDF argument at position 9. - * @tparam A10 type of the UDF argument at position 10. - * @tparam A11 type of the UDF argument at position 11. - * @tparam A12 type of the UDF argument at position 12. - * @tparam A13 type of the UDF argument at position 13. - * @tparam A14 type of the UDF argument at position 14. - * @tparam A15 type of the UDF argument at position 15. - * @tparam A16 type of the UDF argument at position 16. - * @tparam A17 type of the UDF argument at position 17. - * @tparam A18 type of the UDF argument at position 18. - */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag](name: String, func: Function18[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag](name: String, func: Function18[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 19 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. - * @tparam A5 type of the UDF argument at position 5. - * @tparam A6 type of the UDF argument at position 6. - * @tparam A7 type of the UDF argument at position 7. - * @tparam A8 type of the UDF argument at position 8. - * @tparam A9 type of the UDF argument at position 9. - * @tparam A10 type of the UDF argument at position 10. - * @tparam A11 type of the UDF argument at position 11. - * @tparam A12 type of the UDF argument at position 12. - * @tparam A13 type of the UDF argument at position 13. - * @tparam A14 type of the UDF argument at position 14. - * @tparam A15 type of the UDF argument at position 15. - * @tparam A16 type of the UDF argument at position 16. - * @tparam A17 type of the UDF argument at position 17. - * @tparam A18 type of the UDF argument at position 18. - * @tparam A19 type of the UDF argument at position 19. - */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag](name: String, func: Function19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag](name: String, func: Function19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 20 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. - * @tparam A5 type of the UDF argument at position 5. - * @tparam A6 type of the UDF argument at position 6. - * @tparam A7 type of the UDF argument at position 7. - * @tparam A8 type of the UDF argument at position 8. - * @tparam A9 type of the UDF argument at position 9. - * @tparam A10 type of the UDF argument at position 10. - * @tparam A11 type of the UDF argument at position 11. - * @tparam A12 type of the UDF argument at position 12. - * @tparam A13 type of the UDF argument at position 13. - * @tparam A14 type of the UDF argument at position 14. - * @tparam A15 type of the UDF argument at position 15. - * @tparam A16 type of the UDF argument at position 16. - * @tparam A17 type of the UDF argument at position 17. - * @tparam A18 type of the UDF argument at position 18. - * @tparam A19 type of the UDF argument at position 19. - * @tparam A20 type of the UDF argument at position 20. - */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag](name: String, func: Function20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag](name: String, func: Function20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 21 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. - * @tparam A5 type of the UDF argument at position 5. - * @tparam A6 type of the UDF argument at position 6. - * @tparam A7 type of the UDF argument at position 7. - * @tparam A8 type of the UDF argument at position 8. - * @tparam A9 type of the UDF argument at position 9. - * @tparam A10 type of the UDF argument at position 10. - * @tparam A11 type of the UDF argument at position 11. - * @tparam A12 type of the UDF argument at position 12. - * @tparam A13 type of the UDF argument at position 13. - * @tparam A14 type of the UDF argument at position 14. - * @tparam A15 type of the UDF argument at position 15. - * @tparam A16 type of the UDF argument at position 16. - * @tparam A17 type of the UDF argument at position 17. - * @tparam A18 type of the UDF argument at position 18. - * @tparam A19 type of the UDF argument at position 19. - * @tparam A20 type of the UDF argument at position 20. - * @tparam A21 type of the UDF argument at position 21. - */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag](name: String, func: Function21[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag](name: String, func: Function21[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } /** * Register a Scala closure of 22 arguments as user-defined function (UDF). * @tparam RT return type of UDF. - * @tparam A1 type of the UDF argument at position 1. - * @tparam A2 type of the UDF argument at position 2. - * @tparam A3 type of the UDF argument at position 3. - * @tparam A4 type of the UDF argument at position 4. - * @tparam A5 type of the UDF argument at position 5. - * @tparam A6 type of the UDF argument at position 6. - * @tparam A7 type of the UDF argument at position 7. - * @tparam A8 type of the UDF argument at position 8. - * @tparam A9 type of the UDF argument at position 9. - * @tparam A10 type of the UDF argument at position 10. - * @tparam A11 type of the UDF argument at position 11. - * @tparam A12 type of the UDF argument at position 12. - * @tparam A13 type of the UDF argument at position 13. - * @tparam A14 type of the UDF argument at position 14. - * @tparam A15 type of the UDF argument at position 15. - * @tparam A16 type of the UDF argument at position 16. - * @tparam A17 type of the UDF argument at position 17. - * @tparam A18 type of the UDF argument at position 18. - * @tparam A19 type of the UDF argument at position 19. - * @tparam A20 type of the UDF argument at position 20. - * @tparam A21 type of the UDF argument at position 21. - * @tparam A22 type of the UDF argument at position 22. - */ - def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag, A22: TypeTag](name: String, func: Function22[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, RT]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[RT].dataType, e) + */ + def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag, A22: TypeTag](name: String, func: Function22[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, RT]): UserDefinedFunction = { + val dataType = ScalaReflection.schemaFor[RT].dataType + def builder(e: Seq[Expression]) = ScalaUdf(func, dataType, e) functionRegistry.registerFunction(name, builder) + UserDefinedFunction(func, dataType) } + ////////////////////////////////////////////////////////////////////////////////////////////// + ////////////////////////////////////////////////////////////////////////////////////////////// + /** * Register a user-defined function with 1 arguments. */ 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 new file mode 100644 index 0000000000000..8d7c2a1b8339e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/UserDefinedFunction.scala @@ -0,0 +1,39 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql + +import org.apache.spark.sql.catalyst.expressions.ScalaUdf +import org.apache.spark.sql.types.DataType + +/** + * A user-defined function. To create one, use the `udf` functions in [[Dsl]]. + * As an example: + * {{{ + * // Defined a UDF that returns true or false based on some numeric score. + * val predict = udf((score: Double) => if (score > 0.5) true else false) + * + * // Projects a column that adds a prediction column based on the score column. + * df.select( predict(df("score")) ) + * }}} + */ +case class UserDefinedFunction(f: AnyRef, dataType: DataType) { + + def apply(exprs: Column*): Column = { + Column(ScalaUdf(f, dataType, exprs.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 f6b65a81ce05e..19d4f34e565e9 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.Dsl._ import org.apache.spark.sql.types._ /* Implicits */ -import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.test.TestSQLContext.{createDataFrame, logicalPlanToSparkQuery} import scala.language.postfixOps @@ -280,11 +280,11 @@ class DataFrameSuite extends QueryTest { } test("udf") { - val foo = (a: Int, b: String) => a.toString + b + val foo = udf((a: Int, b: String) => a.toString + b) checkAnswer( // SELECT *, foo(key, value) FROM testData - testData.select($"*", callUDF(foo, 'key, 'value)).limit(3), + testData.select($"*", foo('key, 'value)).limit(3), Row(1, "1", "11") :: Row(2, "2", "22") :: Row(3, "3", "33") :: 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 0501b47f080d7..8f3d4265a286d 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,6 +19,7 @@ package org.apache.spark.sql import java.util.TimeZone +import org.apache.spark.sql.test.TestSQLContext import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.Dsl._ @@ -26,9 +27,8 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.types._ -/* Implicits */ import org.apache.spark.sql.TestData._ -import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.test.TestSQLContext.{udf => _, _} class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { @@ -794,7 +794,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("SPARK-3371 Renaming a function expression with group by gives error") { - udf.register("len", (s: String) => s.length) + TestSQLContext.udf.register("len", (s: String) => s.length) checkAnswer( sql("SELECT len(value) as temp FROM testData WHERE key = 1 group by len(value)"), Row(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 0696a2335e63f..117a51173481d 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 @@ -21,7 +21,8 @@ import scala.beans.{BeanInfo, BeanProperty} import org.apache.spark.rdd.RDD import org.apache.spark.sql.Dsl._ -import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.test.TestSQLContext.{udf => _, _} import org.apache.spark.sql.types._ @@ -83,7 +84,7 @@ class UserDefinedTypeSuite extends QueryTest { } test("UDTs and UDFs") { - udf.register("testType", (d: MyDenseVector) => d.isInstanceOf[MyDenseVector]) + TestSQLContext.udf.register("testType", (d: MyDenseVector) => d.isInstanceOf[MyDenseVector]) pointsRDD.registerTempTable("points") checkAnswer( sql("SELECT testType(features) from points"), From d37978d8aafef8a2e637687f3848ca0a8b935b33 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 3 Feb 2015 20:17:12 -0800 Subject: [PATCH 054/578] [SPARK-4795][Core] Redesign the "primitive type => Writable" implicit APIs to make them be activated automatically Try to redesign the "primitive type => Writable" implicit APIs to make them be activated automatically and without breaking binary compatibility. However, this PR will breaking the source compatibility if people use `xxxToXxxWritable` occasionally. See the unit test in `graphx`. Author: zsxwing Closes #3642 from zsxwing/SPARK-4795 and squashes the following commits: 914b2d6 [zsxwing] Add implicit back to the Writables methods 0b9017f [zsxwing] Add some docs a0e8509 [zsxwing] Merge branch 'master' into SPARK-4795 39343de [zsxwing] Fix the unit test 64853af [zsxwing] Reorganize the rest 'implicit' methods in SparkContext --- .../scala/org/apache/spark/SparkContext.scala | 70 ++++++++++++++++++- .../WriteInputFormatTestDataGenerator.scala | 1 - .../main/scala/org/apache/spark/package.scala | 3 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 22 +++--- .../spark/rdd/SequenceFileRDDFunctions.scala | 49 +++++++++---- .../scala/org/apache/spark/FileSuite.scala | 1 - .../org/apache/sparktest/ImplicitSuite.scala | 14 +++- .../spark/graphx/lib/ShortestPathsSuite.scala | 2 +- 8 files changed, 129 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 6a16a31654630..16c6fdbe5274d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1749,8 +1749,14 @@ 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 rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( - rdd: RDD[(K, V)]) = + rdd: RDD[(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 + implicit val keyWritableFactory = new WritableFactory[K](_ => null, kf) + implicit val valueWritableFactory = new WritableFactory[V](_ => null, vf) RDD.rddToSequenceFileRDDFunctions(rdd) + } @deprecated("Replaced by implicit functions in the RDD companion object. This is " + "kept here only for backward compatibility.", "1.3.0") @@ -1767,20 +1773,35 @@ object SparkContext extends Logging { def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) = RDD.numericRDDToDoubleRDDFunctions(rdd) - // Implicit conversions to common Writable types, for saveAsSequenceFile + // The following deprecated functions have already been moved to `object WritableFactory` to + // make the compiler find them automatically. They are still kept here for backward compatibility. + @deprecated("Replaced by implicit functions in the WritableFactory companion object. This is " + + "kept here only for backward compatibility.", "1.3.0") implicit def intToIntWritable(i: Int): IntWritable = new IntWritable(i) + @deprecated("Replaced by implicit functions in the WritableFactory companion object. This is " + + "kept here only for backward compatibility.", "1.3.0") implicit def longToLongWritable(l: Long): LongWritable = new LongWritable(l) + @deprecated("Replaced by implicit functions in the WritableFactory companion object. This is " + + "kept here only for backward compatibility.", "1.3.0") implicit def floatToFloatWritable(f: Float): FloatWritable = new FloatWritable(f) + @deprecated("Replaced by implicit functions in the WritableFactory companion object. This is " + + "kept here only for backward compatibility.", "1.3.0") implicit def doubleToDoubleWritable(d: Double): DoubleWritable = new DoubleWritable(d) + @deprecated("Replaced by implicit functions in the WritableFactory companion object. This is " + + "kept here only for backward compatibility.", "1.3.0") implicit def boolToBoolWritable (b: Boolean): BooleanWritable = new BooleanWritable(b) + @deprecated("Replaced by implicit functions in the WritableFactory companion object. This is " + + "kept here only for backward compatibility.", "1.3.0") implicit def bytesToBytesWritable (aob: Array[Byte]): BytesWritable = new BytesWritable(aob) + @deprecated("Replaced by implicit functions in the WritableFactory companion object. This is " + + "kept here only for backward compatibility.", "1.3.0") implicit def stringToText(s: String): Text = new Text(s) private implicit def arrayToArrayWritable[T <% Writable: ClassTag](arr: Traversable[T]) @@ -2070,7 +2091,7 @@ object WritableConverter { new WritableConverter[T](_ => wClass, x => convert(x.asInstanceOf[W])) } - // The following implicit functions were in SparkContext before 1.2 and users had to + // The following implicit functions were in SparkContext before 1.3 and users had to // `import SparkContext._` to enable them. Now we move them here to make the compiler find // them automatically. However, we still keep the old functions in SparkContext for backward // compatibility and forward to the following functions directly. @@ -2103,3 +2124,46 @@ object WritableConverter { implicit def writableWritableConverter[T <: Writable](): WritableConverter[T] = new WritableConverter[T](_.runtimeClass.asInstanceOf[Class[T]], _.asInstanceOf[T]) } + +/** + * A class encapsulating how to convert some type T to Writable. It stores both the Writable class + * corresponding to T (e.g. IntWritable for Int) and a function for doing the conversion. + * The Writable class will be used in `SequenceFileRDDFunctions`. + */ +private[spark] class WritableFactory[T]( + val writableClass: ClassTag[T] => Class[_ <: Writable], + val convert: T => Writable) extends Serializable + +object WritableFactory { + + private[spark] def simpleWritableFactory[T: ClassTag, W <: Writable : ClassTag](convert: T => W) + : WritableFactory[T] = { + val writableClass = implicitly[ClassTag[W]].runtimeClass.asInstanceOf[Class[W]] + new WritableFactory[T](_ => writableClass, convert) + } + + implicit def intWritableFactory: WritableFactory[Int] = + simpleWritableFactory(new IntWritable(_)) + + implicit def longWritableFactory: WritableFactory[Long] = + simpleWritableFactory(new LongWritable(_)) + + implicit def floatWritableFactory: WritableFactory[Float] = + simpleWritableFactory(new FloatWritable(_)) + + implicit def doubleWritableFactory: WritableFactory[Double] = + simpleWritableFactory(new DoubleWritable(_)) + + implicit def booleanWritableFactory: WritableFactory[Boolean] = + simpleWritableFactory(new BooleanWritable(_)) + + implicit def bytesWritableFactory: WritableFactory[Array[Byte]] = + simpleWritableFactory(new BytesWritable(_)) + + implicit def stringWritableFactory: WritableFactory[String] = + simpleWritableFactory(new Text(_)) + + implicit def writableWritableFactory[T <: Writable: ClassTag]: WritableFactory[T] = + simpleWritableFactory(w => w) + +} 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 c0cbd28a845be..cf289fb3ae39f 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 @@ -107,7 +107,6 @@ private[python] class WritableToDoubleArrayConverter extends Converter[Any, Arra * given directory (probably a temp directory) */ object WriteInputFormatTestDataGenerator { - import SparkContext._ def main(args: Array[String]) { val path = args(0) diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 5ad73c3d27f47..b6249b492150a 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -27,8 +27,7 @@ package org.apache * contains operations available only on RDDs of Doubles; and * [[org.apache.spark.rdd.SequenceFileRDDFunctions]] contains operations available on RDDs that can * be saved as SequenceFiles. These operations are automatically available on any RDD of the right - * type (e.g. RDD[(Int, Int)] through implicit conversions except `saveAsSequenceFile`. You need to - * `import org.apache.spark.SparkContext._` to make `saveAsSequenceFile` work. + * type (e.g. RDD[(Int, Int)] through implicit conversions. * * Java programmers should reference the [[org.apache.spark.api.java]] package * for Spark programming APIs in Java. 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 97aee58bddbf1..fe55a5124f3b6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -25,11 +25,8 @@ import scala.language.implicitConversions import scala.reflect.{classTag, ClassTag} import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus -import org.apache.hadoop.io.BytesWritable +import org.apache.hadoop.io.{Writable, BytesWritable, NullWritable, Text} import org.apache.hadoop.io.compress.CompressionCodec -import org.apache.hadoop.io.NullWritable -import org.apache.hadoop.io.Text -import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.TextOutputFormat import org.apache.spark._ @@ -57,8 +54,7 @@ import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, Bernoulli * [[org.apache.spark.rdd.SequenceFileRDDFunctions]] contains operations available on RDDs that * can be saved as SequenceFiles. * All operations are automatically available on any RDD of the right type (e.g. RDD[(Int, Int)] - * through implicit conversions except `saveAsSequenceFile`. You need to - * `import org.apache.spark.SparkContext._` to make `saveAsSequenceFile` work. + * through implicit. * * Internally, each RDD is characterized by five main properties: * @@ -1527,7 +1523,7 @@ abstract class RDD[T: ClassTag]( */ object RDD { - // The following implicit functions were in SparkContext before 1.2 and users had to + // The following implicit functions were in SparkContext before 1.3 and users had to // `import SparkContext._` to enable them. Now we move them here to make the compiler find // them automatically. However, we still keep the old functions in SparkContext for backward // compatibility and forward to the following functions directly. @@ -1541,9 +1537,15 @@ object RDD { new AsyncRDDActions(rdd) } - implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( - rdd: RDD[(K, V)]): SequenceFileRDDFunctions[K, V] = { - new SequenceFileRDDFunctions(rdd) + implicit def rddToSequenceFileRDDFunctions[K, V](rdd: RDD[(K, V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], + keyWritableFactory: WritableFactory[K], + valueWritableFactory: WritableFactory[V]) + : SequenceFileRDDFunctions[K, V] = { + implicit val keyConverter = keyWritableFactory.convert + implicit val valueConverter = valueWritableFactory.convert + new SequenceFileRDDFunctions(rdd, + keyWritableFactory.writableClass(kt), valueWritableFactory.writableClass(vt)) } implicit def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag](rdd: RDD[(K, V)]) diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala index 2b48916951430..059f8963691f0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala @@ -30,13 +30,35 @@ import org.apache.spark.Logging * through an implicit conversion. Note that this can't be part of PairRDDFunctions because * we need more implicit parameters to convert our keys and values to Writable. * - * Import `org.apache.spark.SparkContext._` at the top of their program to use these functions. */ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag]( - self: RDD[(K, V)]) + self: RDD[(K, V)], + _keyWritableClass: Class[_ <: Writable], + _valueWritableClass: Class[_ <: Writable]) extends Logging with Serializable { + @deprecated("It's used to provide backward compatibility for pre 1.3.0.", "1.3.0") + def this(self: RDD[(K, V)]) { + this(self, null, null) + } + + private val keyWritableClass = + if (_keyWritableClass == null) { + // pre 1.3.0, we need to use Reflection to get the Writable class + getWritableClass[K]() + } else { + _keyWritableClass + } + + private val valueWritableClass = + if (_valueWritableClass == null) { + // pre 1.3.0, we need to use Reflection to get the Writable class + getWritableClass[V]() + } else { + _valueWritableClass + } + private def getWritableClass[T <% Writable: ClassTag](): Class[_ <: Writable] = { val c = { if (classOf[Writable].isAssignableFrom(classTag[T].runtimeClass)) { @@ -55,6 +77,7 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag c.asInstanceOf[Class[_ <: Writable]] } + /** * Output the RDD as a Hadoop SequenceFile using the Writable types we infer from the RDD's key * and value types. If the key or value are Writable, then we use their classes directly; @@ -65,26 +88,28 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag def saveAsSequenceFile(path: String, codec: Option[Class[_ <: CompressionCodec]] = None) { def anyToWritable[U <% Writable](u: U): Writable = u - val keyClass = getWritableClass[K] - val valueClass = getWritableClass[V] - val convertKey = !classOf[Writable].isAssignableFrom(self.keyClass) - val convertValue = !classOf[Writable].isAssignableFrom(self.valueClass) + // TODO We cannot force the return type of `anyToWritable` be same as keyWritableClass and + // valueWritableClass at the compile time. To implement that, we need to add type parameters to + // SequenceFileRDDFunctions. however, SequenceFileRDDFunctions is a public class so it will be a + // breaking change. + val convertKey = self.keyClass != keyWritableClass + val convertValue = self.valueClass != valueWritableClass - logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," + - valueClass.getSimpleName + ")" ) + logInfo("Saving as sequence file of type (" + keyWritableClass.getSimpleName + "," + + valueWritableClass.getSimpleName + ")" ) val format = classOf[SequenceFileOutputFormat[Writable, Writable]] val jobConf = new JobConf(self.context.hadoopConfiguration) if (!convertKey && !convertValue) { - self.saveAsHadoopFile(path, keyClass, valueClass, format, jobConf, codec) + self.saveAsHadoopFile(path, keyWritableClass, valueWritableClass, format, jobConf, codec) } else if (!convertKey && convertValue) { self.map(x => (x._1,anyToWritable(x._2))).saveAsHadoopFile( - path, keyClass, valueClass, format, jobConf, codec) + path, keyWritableClass, valueWritableClass, format, jobConf, codec) } else if (convertKey && !convertValue) { self.map(x => (anyToWritable(x._1),x._2)).saveAsHadoopFile( - path, keyClass, valueClass, format, jobConf, codec) + path, keyWritableClass, valueWritableClass, format, jobConf, codec) } else if (convertKey && convertValue) { self.map(x => (anyToWritable(x._1),anyToWritable(x._2))).saveAsHadoopFile( - path, keyClass, valueClass, format, jobConf, codec) + path, keyWritableClass, valueWritableClass, format, jobConf, codec) } } } diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 5e24196101fbc..7acd27c735727 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -32,7 +32,6 @@ import org.apache.hadoop.mapreduce.lib.input.{FileSplit => NewFileSplit, TextInp import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} import org.scalatest.FunSuite -import org.apache.spark.SparkContext._ import org.apache.spark.rdd.{NewHadoopRDD, HadoopRDD} import org.apache.spark.util.Utils diff --git a/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala b/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala index 4918e2d92beb4..daa795a043495 100644 --- a/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala +++ b/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala @@ -44,13 +44,21 @@ class ImplicitSuite { } def testRddToSequenceFileRDDFunctions(): Unit = { - // TODO eliminating `import intToIntWritable` needs refactoring SequenceFileRDDFunctions. - // That will be a breaking change. - import org.apache.spark.SparkContext.intToIntWritable val rdd: org.apache.spark.rdd.RDD[(Int, Int)] = mockRDD rdd.saveAsSequenceFile("/a/test/path") } + def testRddToSequenceFileRDDFunctionsWithWritable(): Unit = { + val rdd: org.apache.spark.rdd.RDD[(org.apache.hadoop.io.IntWritable, org.apache.hadoop.io.Text)] + = mockRDD + rdd.saveAsSequenceFile("/a/test/path") + } + + def testRddToSequenceFileRDDFunctionsWithBytesArray(): Unit = { + val rdd: org.apache.spark.rdd.RDD[(Int, Array[Byte])] = mockRDD + rdd.saveAsSequenceFile("/a/test/path") + } + def testRddToOrderedRDDFunctions(): Unit = { val rdd: org.apache.spark.rdd.RDD[(Int, Int)] = mockRDD rdd.sortByKey() diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala index 265827b3341c2..f2c38e79c452c 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/ShortestPathsSuite.scala @@ -40,7 +40,7 @@ class ShortestPathsSuite extends FunSuite with LocalSparkContext { val graph = Graph.fromEdgeTuples(edges, 1) val landmarks = Seq(1, 4).map(_.toLong) val results = ShortestPaths.run(graph, landmarks).vertices.collect.map { - case (v, spMap) => (v, spMap.mapValues(_.get)) + case (v, spMap) => (v, spMap.mapValues(i => i)) } assert(results.toSet === shortestPaths) } From eb15631854f8e8e7bcd1b3331f18d6ee81dd6d5c Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 3 Feb 2015 20:39:11 -0800 Subject: [PATCH 055/578] [FIX][MLLIB] fix seed handling in Python GMM If `seed` is `None` on the python side, it will pass in as a `null`. So we should use `java.lang.Long` instead of `Long` to take it. Author: Xiangrui Meng Closes #4349 from mengxr/gmm-fix and squashes the following commits: 3be5926 [Xiangrui Meng] fix seed handling in Python GMM --- .../org/apache/spark/mllib/api/python/PythonMLLibAPI.scala | 2 +- 1 file changed, 1 insertion(+), 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 980980593d194..3f29b82ce85ac 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 @@ -295,7 +295,7 @@ class PythonMLLibAPI extends Serializable { k: Int, convergenceTol: Double, maxIterations: Int, - seed: Long): JList[Object] = { + seed: java.lang.Long): JList[Object] = { val gmmAlg = new GaussianMixture() .setK(k) .setConvergenceTol(convergenceTol) From 40c4cb2fe79ceac0d656be7b72cb2ee8d7db7258 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 3 Feb 2015 22:15:35 -0800 Subject: [PATCH 056/578] [SPARK-5579][SQL][DataFrame] Support for project/filter using SQL expressions ```scala df.selectExpr("abs(colA)", "colB") df.filter("age > 21") ``` Author: Reynold Xin Closes #4348 from rxin/SPARK-5579 and squashes the following commits: 2baeef2 [Reynold Xin] Fix Python. b416372 [Reynold Xin] [SPARK-5579][SQL][DataFrame] Support for project/filter using SQL expressions. --- python/pyspark/sql.py | 5 ++-- .../apache/spark/sql/catalyst/SqlParser.scala | 10 ++++++++ .../org/apache/spark/sql/DataFrame.scala | 23 +++++++++++++++++-- .../org/apache/spark/sql/DataFrameImpl.scala | 22 +++++++++++++----- .../apache/spark/sql/IncomputableColumn.scala | 8 +++++-- .../org/apache/spark/sql/DataFrameSuite.scala | 12 ++++++++++ 6 files changed, 67 insertions(+), 13 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 268c7ef97cffc..74305dea749c8 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -2126,10 +2126,9 @@ def sort(self, *cols): """ if not cols: raise ValueError("should sort by at least one column") - jcols = ListConverter().convert([_to_java_column(c) for c in cols[1:]], + jcols = ListConverter().convert([_to_java_column(c) for c in cols], self._sc._gateway._gateway_client) - jdf = self._jdf.sort(_to_java_column(cols[0]), - self._sc._jvm.Dsl.toColumns(jcols)) + jdf = self._jdf.sort(self._sc._jvm.Dsl.toColumns(jcols)) return DataFrame(jdf, self.sql_ctx) sortBy = sort 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 5c006e9d4c6f5..a9bd079c7049d 100755 --- 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 @@ -36,6 +36,16 @@ import org.apache.spark.sql.types._ * for a SQL like language should checkout the HiveQL support in the sql/hive sub-project. */ class SqlParser extends AbstractSparkSQLParser { + + def parseExpression(input: String): Expression = { + // Initialize the Keywords. + lexical.initialize(reservedWords) + phrase(expression)(new lexical.Scanner(input)) match { + case Success(plan, _) => plan + case failureOrError => sys.error(failureOrError.toString) + } + } + // 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 ABS = Keyword("ABS") 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 732b6855587dd..a4997fb293781 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 @@ -173,7 +173,7 @@ trait DataFrame extends RDDApi[Row] { * }}} */ @scala.annotation.varargs - def sort(sortExpr: Column, sortExprs: Column*): DataFrame + def sort(sortExprs: Column*): DataFrame /** * Returns a new [[DataFrame]] sorted by the given expressions. @@ -187,7 +187,7 @@ trait DataFrame extends RDDApi[Row] { * This is an alias of the `sort` function. */ @scala.annotation.varargs - def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame + def orderBy(sortExprs: Column*): DataFrame /** * Selects column based on the column name and return it as a [[Column]]. @@ -236,6 +236,17 @@ trait DataFrame extends RDDApi[Row] { @scala.annotation.varargs def select(col: String, cols: String*): DataFrame + /** + * Selects a set of SQL expressions. This is a variant of `select` that accepts + * SQL expressions. + * + * {{{ + * df.selectExpr("colA", "colB as newName", "abs(colC)") + * }}} + */ + @scala.annotation.varargs + def selectExpr(exprs: String*): DataFrame + /** * Filters rows using the given condition. * {{{ @@ -247,6 +258,14 @@ trait DataFrame extends RDDApi[Row] { */ def filter(condition: Column): DataFrame + /** + * Filters rows using the given SQL expression. + * {{{ + * peopleDf.filter("age > 15") + * }}} + */ + def filter(conditionExpr: String): DataFrame + /** * Filters rows using the given condition. This is an alias for `filter`. * {{{ 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 a52bfa59a1365..c702adcb65122 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 @@ -27,7 +27,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 +import org.apache.spark.sql.catalyst.{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} @@ -124,11 +124,11 @@ private[sql] class DataFrameImpl protected[sql]( } override def sort(sortCol: String, sortCols: String*): DataFrame = { - orderBy(apply(sortCol), sortCols.map(apply) :_*) + sort((sortCol +: sortCols).map(apply) :_*) } - override def sort(sortExpr: Column, sortExprs: Column*): DataFrame = { - val sortOrder: Seq[SortOrder] = (sortExpr +: sortExprs).map { col => + override def sort(sortExprs: Column*): DataFrame = { + val sortOrder: Seq[SortOrder] = sortExprs.map { col => col.expr match { case expr: SortOrder => expr @@ -143,8 +143,8 @@ private[sql] class DataFrameImpl protected[sql]( sort(sortCol, sortCols :_*) } - override def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame = { - sort(sortExpr, sortExprs :_*) + override def orderBy(sortExprs: Column*): DataFrame = { + sort(sortExprs :_*) } override def col(colName: String): Column = colName match { @@ -179,10 +179,20 @@ private[sql] class DataFrameImpl protected[sql]( select((col +: cols).map(Column(_)) :_*) } + override def selectExpr(exprs: String*): DataFrame = { + select(exprs.map { expr => + Column(new SqlParser().parseExpression(expr)) + } :_*) + } + 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) } 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 ba5c7355b4b70..6b032d3d699a9 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 @@ -66,11 +66,11 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten override def sort(sortCol: String, sortCols: String*): DataFrame = err() - override def sort(sortExpr: Column, sortExprs: Column*): DataFrame = err() + override def sort(sortExprs: Column*): DataFrame = err() override def orderBy(sortCol: String, sortCols: String*): DataFrame = err() - override def orderBy(sortExpr: Column, sortExprs: Column*): DataFrame = err() + override def orderBy(sortExprs: Column*): DataFrame = err() override def col(colName: String): Column = err() @@ -80,8 +80,12 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten override def select(col: String, cols: String*): DataFrame = err() + override def selectExpr(exprs: 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 apply(condition: Column): DataFrame = 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 19d4f34e565e9..e588555ad01d4 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 @@ -47,6 +47,18 @@ class DataFrameSuite extends QueryTest { testData.collect().toSeq) } + test("selectExpr") { + checkAnswer( + testData.selectExpr("abs(key)", "value"), + testData.collect().map(row => Row(math.abs(row.getInt(0)), row.getString(1))).toSeq) + } + + test("filterExpr") { + checkAnswer( + testData.filter("key > 90"), + testData.collect().filter(_.getInt(0) > 90).toSeq) + } + test("repartition") { checkAnswer( testData.select('key).repartition(10).select('key), From 242b4f02df7f71ebcfa86a85c9ed39e40750a7fd Mon Sep 17 00:00:00 2001 From: freeman Date: Tue, 3 Feb 2015 22:24:30 -0800 Subject: [PATCH 057/578] [SPARK-4969][STREAMING][PYTHON] Add binaryRecords to streaming In Spark 1.2 we added a `binaryRecords` input method for loading flat binary data. This format is useful for numerical array data, e.g. in scientific computing applications. This PR adds support for the same format in Streaming applications, where it is similarly useful, especially for streaming time series or sensor data. Summary of additions - adding `binaryRecordsStream` to Spark Streaming - exposing `binaryRecordsStream` in the new PySpark Streaming - new unit tests in Scala and Python This required adding an optional Hadoop configuration param to `fileStream` and `FileInputStream`, but was otherwise straightforward. tdas davies Author: freeman Closes #3803 from freeman-lab/streaming-binary-records and squashes the following commits: b676534 [freeman] Clarify note 5ff1b75 [freeman] Add note to java streaming context eba925c [freeman] Simplify notes c4237b8 [freeman] Add experimental tag 30eba67 [freeman] Add filter and newFilesOnly alongside conf c2cfa6d [freeman] Expose new version of fileStream with conf in java 34d20ef [freeman] Add experimental tag 14bca9a [freeman] Add experimental tag b85bffc [freeman] Formatting 47560f4 [freeman] Space formatting 9a3715a [freeman] Refactor to reflect changes to FileInputSuite 7373f73 [freeman] Add note and defensive assertion for byte length 3ceb684 [freeman] Merge remote-tracking branch 'upstream/master' into streaming-binary-records 317b6d1 [freeman] Make test inline fcb915c [freeman] Formatting becb344 [freeman] Formatting d3e75b2 [freeman] Add tests in python a4324a3 [freeman] Line length 029d49c [freeman] Formatting 1c739aa [freeman] Simpler default arg handling 94d90d0 [freeman] Spelling 2843e9d [freeman] Add params to docstring 8b70fbc [freeman] Reorganization 28bff9b [freeman] Fix missing arg 9398bcb [freeman] Expose optional hadoop configuration 23dd69f [freeman] Tests for binaryRecordsStream 36cb0fd [freeman] Add binaryRecordsStream to scala fe4e803 [freeman] Add binaryRecordStream to Java API ecef0eb [freeman] Add binaryRecordsStream to python 8550c26 [freeman] Expose additional argument combination --- .../scala/org/apache/spark/SparkContext.scala | 9 ++- python/pyspark/streaming/context.py | 16 ++++- python/pyspark/streaming/tests.py | 15 +++++ .../spark/streaming/StreamingContext.scala | 59 ++++++++++++++++++- .../api/java/JavaStreamingContext.scala | 52 +++++++++++++++- .../streaming/dstream/FileInputDStream.scala | 17 +++++- .../spark/streaming/InputStreamsSuite.scala | 51 ++++++++++++++++ 7 files changed, 212 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 16c6fdbe5274d..eecfdd4222adf 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -657,6 +657,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * * Load data from a flat binary file, assuming the length of each record is constant. * + * '''Note:''' We ensure that the byte array for each record in the resulting RDD + * has the provided record length. + * * @param path Directory to the input data files * @param recordLength The length at which to split the records * @return An RDD of data with values, represented as byte arrays @@ -671,7 +674,11 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli classOf[LongWritable], classOf[BytesWritable], conf=conf) - val data = br.map{ case (k, v) => v.getBytes} + val data = br.map { case (k, v) => + val bytes = v.getBytes + assert(bytes.length == recordLength, "Byte array does not have correct length") + bytes + } data } diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py index d48f3598e33b2..18aaae93b05f2 100644 --- a/python/pyspark/streaming/context.py +++ b/python/pyspark/streaming/context.py @@ -21,7 +21,7 @@ from py4j.java_gateway import java_import, JavaObject from pyspark import RDD, SparkConf -from pyspark.serializers import UTF8Deserializer, CloudPickleSerializer +from pyspark.serializers import NoOpSerializer, UTF8Deserializer, CloudPickleSerializer from pyspark.context import SparkContext from pyspark.storagelevel import StorageLevel from pyspark.streaming.dstream import DStream @@ -251,6 +251,20 @@ def textFileStream(self, directory): """ return DStream(self._jssc.textFileStream(directory), self, UTF8Deserializer()) + def binaryRecordsStream(self, directory, recordLength): + """ + Create an input stream that monitors a Hadoop-compatible file system + for new files and reads them as flat binary files with records of + fixed length. Files must be written to the monitored directory by "moving" + them from another location within the same file system. + File names starting with . are ignored. + + @param directory: Directory to load data from + @param recordLength: Length of each record in bytes + """ + return DStream(self._jssc.binaryRecordsStream(directory, recordLength), self, + NoOpSerializer()) + def _check_serializers(self, rdds): # make sure they have same serializer if len(set(rdd._jrdd_deserializer for rdd in rdds)) > 1: diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index a8d876d0fa3b3..608f8e26473a6 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -21,6 +21,7 @@ import operator import unittest import tempfile +import struct from pyspark.context import SparkConf, SparkContext, RDD from pyspark.streaming.context import StreamingContext @@ -455,6 +456,20 @@ def test_text_file_stream(self): self.wait_for(result, 2) self.assertEqual([range(10), 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))) + result = self._collect(dstream, 2, block=False) + self.ssc.start() + for name in ('a', 'b'): + time.sleep(1) + 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)) + def test_union(self): input = [range(i + 1) for i in range(3)] dstream = self.ssc.queueStream(input) 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 8ef0787137845..ddc435cf1a2e6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -27,10 +27,12 @@ import scala.reflect.ClassTag import akka.actor.{Props, SupervisorStrategy} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.io.{LongWritable, Text} +import org.apache.hadoop.io.{BytesWritable, LongWritable, Text} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat import org.apache.spark._ +import org.apache.spark.annotation.Experimental +import org.apache.spark.input.FixedLengthBinaryInputFormat import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream._ @@ -359,6 +361,30 @@ class StreamingContext private[streaming] ( new FileInputDStream[K, V, F](this, directory, filter, newFilesOnly) } + /** + * Create a input stream that monitors a Hadoop-compatible filesystem + * for new files and reads them using the given key-value types and input format. + * Files must be written to the monitored directory by "moving" them from another + * location within the same file system. File names starting with . are ignored. + * @param directory HDFS directory to monitor for new file + * @param filter Function to filter paths to process + * @param newFilesOnly Should process only new files and ignore existing files in the directory + * @param conf Hadoop configuration + * @tparam K Key type for reading HDFS file + * @tparam V Value type for reading HDFS file + * @tparam F Input format for reading HDFS file + */ + def fileStream[ + K: ClassTag, + V: ClassTag, + F <: NewInputFormat[K, V]: ClassTag + ] (directory: String, + filter: Path => Boolean, + newFilesOnly: Boolean, + conf: Configuration): InputDStream[(K, V)] = { + new FileInputDStream[K, V, F](this, directory, filter, newFilesOnly, Option(conf)) + } + /** * Create a input stream that monitors a Hadoop-compatible filesystem * for new files and reads them as text files (using key as LongWritable, value @@ -371,6 +397,37 @@ class StreamingContext private[streaming] ( fileStream[LongWritable, Text, TextInputFormat](directory).map(_._2.toString) } + /** + * :: Experimental :: + * + * Create an input stream that monitors a Hadoop-compatible filesystem + * for new files and reads them as flat binary files, assuming a fixed length per record, + * generating one byte array per record. Files must be written to the monitored directory + * by "moving" them from another location within the same file system. File names + * starting with . are ignored. + * + * '''Note:''' We ensure that the byte array for each record in the + * resulting RDDs of the DStream has the provided record length. + * + * @param directory HDFS directory to monitor for new file + * @param recordLength length of each record in bytes + */ + @Experimental + def binaryRecordsStream( + directory: String, + recordLength: Int): DStream[Array[Byte]] = { + val conf = sc_.hadoopConfiguration + conf.setInt(FixedLengthBinaryInputFormat.RECORD_LENGTH_PROPERTY, recordLength) + val br = fileStream[LongWritable, BytesWritable, FixedLengthBinaryInputFormat]( + directory, FileInputDStream.defaultFilter : Path => Boolean, newFilesOnly=true, conf) + val data = br.map { case (k, v) => + val bytes = v.getBytes + assert(bytes.length == recordLength, "Byte array does not have correct length") + bytes + } + data + } + /** * Create an input stream from a queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. 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 9a2254bcdc1f7..0f7ae7a1c7de8 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 @@ -29,6 +29,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} import org.apache.spark.rdd.RDD @@ -177,7 +178,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { /** * Create an input stream from network source hostname:port. Data is received using - * a TCP socket and the receive bytes it interepreted as object using the given + * a TCP socket and the receive bytes it interpreted as object using the given * converter. * @param hostname Hostname to connect to for receiving data * @param port Port to connect to for receiving data @@ -209,6 +210,24 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { ssc.textFileStream(directory) } + /** + * :: Experimental :: + * + * Create an input stream that monitors a Hadoop-compatible filesystem + * for new files and reads them as flat binary files with fixed record lengths, + * yielding byte arrays + * + * '''Note:''' We ensure that the byte array for each record in the + * resulting RDDs of the DStream has the provided record length. + * + * @param directory HDFS directory to monitor for new files + * @param recordLength The length at which to split the records + */ + @Experimental + def binaryRecordsStream(directory: String, recordLength: Int): JavaDStream[Array[Byte]] = { + ssc.binaryRecordsStream(directory, recordLength) + } + /** * Create an input stream from network source hostname:port, where data is received * as serialized blocks (serialized using the Spark's serializer) that can be directly @@ -298,6 +317,37 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { ssc.fileStream[K, V, F](directory, fn, newFilesOnly) } + /** + * Create an input stream that monitors a Hadoop-compatible filesystem + * for new files and reads them using the given key-value types and input format. + * Files must be written to the monitored directory by "moving" them from another + * location within the same file system. File names starting with . are ignored. + * @param directory HDFS directory to monitor for new file + * @param kClass class of key for reading HDFS file + * @param vClass class of value for reading HDFS file + * @param fClass class of input format for reading HDFS file + * @param filter Function to filter paths to process + * @param newFilesOnly Should process only new files and ignore existing files in the directory + * @param conf Hadoop configuration + * @tparam K Key type for reading HDFS file + * @tparam V Value type for reading HDFS file + * @tparam F Input format for reading HDFS file + */ + def fileStream[K, V, F <: NewInputFormat[K, V]]( + directory: String, + kClass: Class[K], + vClass: Class[V], + fClass: Class[F], + filter: JFunction[Path, JBoolean], + newFilesOnly: Boolean, + conf: Configuration): JavaPairInputDStream[K, V] = { + 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() + ssc.fileStream[K, V, F](directory, fn, newFilesOnly, conf) + } + /** * Create an input stream with any arbitrary user implemented actor receiver. * @param props Props object defining creation of the actor 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 e7c5639a63499..6379b88527ec8 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 @@ -23,6 +23,7 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable import scala.reflect.ClassTag +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} @@ -68,11 +69,13 @@ import org.apache.spark.util.{TimeStampedHashMap, Utils} * processing semantics are undefined. */ private[streaming] -class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : ClassTag]( +class FileInputDStream[K, V, F <: NewInputFormat[K,V]]( @transient ssc_ : StreamingContext, directory: String, filter: Path => Boolean = FileInputDStream.defaultFilter, - newFilesOnly: Boolean = true) + newFilesOnly: Boolean = true, + conf: Option[Configuration] = None) + (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]) extends InputDStream[(K, V)](ssc_) { // This is a def so that it works during checkpoint recovery: @@ -237,7 +240,15 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas /** Generate one RDD from an array of files */ private def filesToRDD(files: Seq[String]): RDD[(K, V)] = { val fileRDDs = files.map(file =>{ - val rdd = context.sparkContext.newAPIHadoopFile[K, V, F](file) + val rdd = conf match { + case Some(config) => context.sparkContext.newAPIHadoopFile( + file, + fm.runtimeClass.asInstanceOf[Class[F]], + km.runtimeClass.asInstanceOf[Class[K]], + vm.runtimeClass.asInstanceOf[Class[V]], + config) + case None => context.sparkContext.newAPIHadoopFile[K, V, F](file) + } if (rdd.partitions.size == 0) { logError("File " + file + " has no data in it. Spark Streaming can only ingest " + "files that have been \"moved\" to the directory assigned to the file stream. " + 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 bddf51e130422..01084a457db4f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -95,6 +95,57 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } } + test("binary records stream") { + val testDir: File = null + try { + val batchDuration = Seconds(2) + val testDir = Utils.createTempDir() + // Create a file that exists before the StreamingContext is created: + val existingFile = new File(testDir, "0") + Files.write("0\n", existingFile, Charset.forName("UTF-8")) + assert(existingFile.setLastModified(10000) && existingFile.lastModified === 10000) + + // Set up the streaming context and input streams + withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc => + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + // This `setTime` call ensures that the clock is past the creation time of `existingFile` + clock.setTime(existingFile.lastModified + batchDuration.milliseconds) + val batchCounter = new BatchCounter(ssc) + val fileStream = ssc.binaryRecordsStream(testDir.toString, 1) + val outputBuffer = new ArrayBuffer[Seq[Array[Byte]]] + with SynchronizedBuffer[Seq[Array[Byte]]] + val outputStream = new TestOutputStream(fileStream, outputBuffer) + outputStream.register() + ssc.start() + + // Advance the clock so that the files are created after StreamingContext starts, but + // not enough to trigger a batch + clock.addToTime(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) + logInfo("Created file " + file) + // Advance the clock after creating the file to avoid a race when + // setting its modification time + clock.addToTime(batchDuration.milliseconds) + eventually(eventuallyTimeout) { + assert(batchCounter.getNumCompletedBatches === i) + } + } + + val expectedOutput = input.map(i => i.toByte) + val obtainedOutput = outputBuffer.flatten.toList.map(i => i(0).toByte) + assert(obtainedOutput === expectedOutput) + } + } finally { + if (testDir != null) Utils.deleteRecursively(testDir) + } + } test("file input stream - newFilesOnly = true") { testFileStream(newFilesOnly = true) From 83de71c45bb9f22049243dd7518b679c4e13c2df Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 3 Feb 2015 22:30:23 -0800 Subject: [PATCH 058/578] [SPARK-4939] revive offers periodically in LocalBackend The locality timeout assume that the SchedulerBackend can revive offers periodically, but currently LocalBackend did do that, then some job with mixed locality levels in local mode will hang forever. This PR let LocalBackend revive offers periodically, just like in cluster mode. Author: Davies Liu Closes #4147 from davies/revive and squashes the following commits: 2acdf9d [Davies Liu] Update LocalBackend.scala 3c8ca7c [Davies Liu] Update LocalBackend.scala d1b60d2 [Davies Liu] address comments from Kay 33ac9bb [Davies Liu] fix build d0da0d5 [Davies Liu] Merge branch 'master' of github.com:apache/spark into revive 6cf5972 [Davies Liu] fix thread-safety ed62a31 [Davies Liu] fix scala style df9008b [Davies Liu] fix typo bfc1396 [Davies Liu] revive offers periodically in LocalBackend --- .../apache/spark/scheduler/local/LocalBackend.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) 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 05b6fa54564b7..4676b828d3d89 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 @@ -19,6 +19,8 @@ package org.apache.spark.scheduler.local import java.nio.ByteBuffer +import scala.concurrent.duration._ + import akka.actor.{Actor, ActorRef, Props} import org.apache.spark.{Logging, SparkContext, SparkEnv, TaskState} @@ -46,6 +48,8 @@ private[spark] class LocalActor( private val totalCores: Int) extends Actor with ActorLogReceive with Logging { + import context.dispatcher // to use Akka's scheduler.scheduleOnce() + private var freeCores = totalCores private val localExecutorId = SparkContext.DRIVER_IDENTIFIER @@ -74,11 +78,16 @@ private[spark] class LocalActor( def reviveOffers() { val offers = Seq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores)) - for (task <- scheduler.resourceOffers(offers).flatten) { + val tasks = scheduler.resourceOffers(offers).flatten + for (task <- tasks) { freeCores -= scheduler.CPUS_PER_TASK executor.launchTask(executorBackend, taskId = task.taskId, attemptNumber = task.attemptNumber, task.name, task.serializedTask) } + 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) + } } } From 6aed719e503afa48820f6e3e798da483649dfcb9 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Tue, 3 Feb 2015 22:39:17 -0800 Subject: [PATCH 059/578] [SPARK-5341] Use maven coordinates as dependencies in spark-shell and spark-submit This PR adds support for using maven coordinates as dependencies to spark-shell. Coordinates can be provided as a comma-delimited string after the flag `--packages`. Additional remote repositories (like SonaType) can be supplied as a comma-delimited string after the flag `--repositories`. Uses the Ivy library to resolve dependencies. Unfortunately the library has no decent documentation, therefore solving more complex dependency issues can be a problem. pwendell, mateiz, mengxr **Note: This is still a WIP. The following need to be handled:** - [x] add docs for the methods - [x] take local ivy cache path as an argument - [x] add tests - [x] add Windows compatibility - [x] exclude unused Ivy dependencies Author: Burak Yavuz Closes #4215 from brkyvz/SPARK-5341ivy and squashes the following commits: 9215851 [Burak Yavuz] ready to merge db2a5cc [Burak Yavuz] changed logging to printStream 9dae87f [Burak Yavuz] file separators changed 71c374d [Burak Yavuz] merge conflicts fixed c08dc9f [Burak Yavuz] fixed merge conflicts 3ada19a [Burak Yavuz] fixed Jenkins error (hopefully) and added comment on oro 43c2290 [Burak Yavuz] fixed that ONE line 231f72f [Burak Yavuz] addressed code review 2cd6562 [Burak Yavuz] Merge branch 'master' of github.com:apache/spark into SPARK-5341ivy 85ec5a3 [Burak Yavuz] added oro as a dependency explicitly ea44ca4 [Burak Yavuz] add oro back to dependencies cef0e24 [Burak Yavuz] IntelliJ is just messing things up 97c4a92 [Burak Yavuz] fix more weird IntelliJ formatting 9cf077d [Burak Yavuz] fix weird IntelliJ formatting dcf5e13 [Burak Yavuz] fix windows command line flags 3a23f21 [Burak Yavuz] excluded ivy dependencies 53423e0 [Burak Yavuz] tests added 3705907 [Burak Yavuz] remove ivy-repo as a command line argument. Use global ivy cache as default c04d885 [Burak Yavuz] take path to ivy cache as a conf 2edc9b5 [Burak Yavuz] managed to exclude Spark and it's dependencies a0870af [Burak Yavuz] add docs. remove unnecesary new lines 6645af4 [Burak Yavuz] [SPARK-5341] added base implementation 882c4c8 [Burak Yavuz] added maven dependency download --- bin/utils.sh | 6 +- bin/windows-utils.cmd | 2 +- core/pom.xml | 11 + .../scala/org/apache/spark/SparkContext.scala | 14 +- .../org/apache/spark/deploy/SparkSubmit.scala | 217 +++++++++++++++++- .../spark/deploy/SparkSubmitArguments.scala | 21 ++ .../spark/deploy/SparkSubmitSuite.scala | 20 +- .../spark/deploy/SparkSubmitUtilsSuite.scala | 121 ++++++++++ pom.xml | 2 + 9 files changed, 404 insertions(+), 10 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala diff --git a/bin/utils.sh b/bin/utils.sh index 22ea2b9a6d586..2241200082018 100755 --- a/bin/utils.sh +++ b/bin/utils.sh @@ -26,14 +26,14 @@ function gatherSparkSubmitOpts() { exit 1 fi - # NOTE: If you add or remove spark-sumbmit options, + # 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 | --py-files | --files | \ - --conf | --properties-file | --driver-memory | --driver-java-options | \ + --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) if [[ $# -lt 2 ]]; then diff --git a/bin/windows-utils.cmd b/bin/windows-utils.cmd index 1082a952dac99..567b8733f7f77 100644 --- a/bin/windows-utils.cmd +++ b/bin/windows-utils.cmd @@ -32,7 +32,7 @@ SET opts="\<--master\> \<--deploy-mode\> \<--class\> \<--name\> \<--jars\> \<--p 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\>" +SET opts="%opts:~1,-1% \<--num-executors\> \<--archives\> \<--packages\> \<--repositories\>" echo %1 | findstr %opts% >nul if %ERRORLEVEL% equ 0 ( diff --git a/core/pom.xml b/core/pom.xml index 2c115683fce66..2dc5f747f2b71 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -241,6 +241,17 @@ derby test + + org.apache.ivy + ivy + ${ivy.version} + + + oro + + oro + ${oro.version} + org.tachyonproject tachyon-client diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index eecfdd4222adf..7f5aef1c75df2 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1231,7 +1231,19 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli null } } else { - env.httpFileServer.addJar(new File(uri.getPath)) + try { + env.httpFileServer.addJar(new File(uri.getPath)) + } catch { + case exc: FileNotFoundException => + logError(s"Jar not found at $path") + null + case e: Exception => + // For now just log an error but allow to go through so spark examples work. + // The spark examples don't really need the jar distributed since its also + // the app jar. + logError("Error adding jar (" + e + "), was the --addJars option used?") + null + } } // A JAR file which exists locally on every worker node case "local" => 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 02021be9f93d4..8bbfcd2997dc6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -25,6 +25,17 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, Map} import org.apache.hadoop.fs.Path +import org.apache.ivy.Ivy +import org.apache.ivy.core.LogOptions +import org.apache.ivy.core.module.descriptor.{DefaultExcludeRule, DefaultDependencyDescriptor, DefaultModuleDescriptor} +import org.apache.ivy.core.module.id.{ModuleId, ArtifactId, ModuleRevisionId} +import org.apache.ivy.core.report.ResolveReport +import org.apache.ivy.core.resolve.{IvyNode, ResolveOptions} +import org.apache.ivy.core.retrieve.RetrieveOptions +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.executor.ExecutorURLClassLoader import org.apache.spark.util.Utils @@ -194,6 +205,18 @@ 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]( @@ -202,6 +225,7 @@ object SparkSubmit { OptionAssigner(args.master, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.master"), OptionAssigner(args.name, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.app.name"), OptionAssigner(args.jars, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.jars"), + OptionAssigner(args.ivyRepoPath, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.jars.ivy"), OptionAssigner(args.driverMemory, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.driver.memory"), OptionAssigner(args.driverExtraClassPath, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, @@ -213,6 +237,7 @@ object SparkSubmit { // Standalone cluster only OptionAssigner(args.jars, STANDALONE, CLUSTER, sysProp = "spark.jars"), + OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, sysProp = "spark.jars.ivy"), OptionAssigner(args.driverMemory, STANDALONE, CLUSTER, clOption = "--memory"), OptionAssigner(args.driverCores, STANDALONE, CLUSTER, clOption = "--cores"), @@ -384,8 +409,8 @@ object SparkSubmit { case e: ClassNotFoundException => e.printStackTrace(printStream) if (childMainClass.contains("thriftserver")) { - println(s"Failed to load main class $childMainClass.") - println("You need to build Spark with -Phive and -Phive-thriftserver.") + printStream.println(s"Failed to load main class $childMainClass.") + printStream.println("You need to build Spark with -Phive and -Phive-thriftserver.") } System.exit(CLASS_NOT_FOUND_EXIT_STATUS) } @@ -475,6 +500,194 @@ object SparkSubmit { } } +/** Provides utility functions to be used inside SparkSubmit. */ +private[spark] object SparkSubmitUtils { + + // Exposed for testing + private[spark] var printStream = SparkSubmit.printStream + + /** + * Represents a Maven Coordinate + * @param groupId the groupId of the coordinate + * @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) + +/** + * Extracts maven coordinates from a comma-delimited string + * @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(":") + 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 " + + s"be whitespace. The groupId provided is: ${splits(0)}") + require(splits(1) != null && splits(1).trim.nonEmpty, s"The artifactId cannot be null or " + + s"be whitespace. The artifactId provided is: ${splits(1)}") + require(splits(2) != null && splits(2).trim.nonEmpty, s"The version cannot be null or " + + s"be whitespace. The version provided is: ${splits(2)}") + new MavenCoordinate(splits(0), splits(1), splits(2)) + } + } + + /** + * Extracts maven coordinates from a comma-delimited string + * @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 = { + // We need a chain resolver if we want to check multiple repositories + val cr = new ChainResolver + cr.setName("list") + + // the biblio resolver resolves POM declared dependencies + val br: IBiblioResolver = new IBiblioResolver + br.setM2compatible(true) + br.setUsepoms(true) + br.setName("central") + cr.add(br) + + val repositoryList = remoteRepos.getOrElse("") + // add any other remote repositories other than maven central + if (repositoryList.trim.nonEmpty) { + repositoryList.split(",").zipWithIndex.foreach { case (repo, i) => + val brr: IBiblioResolver = new IBiblioResolver + brr.setM2compatible(true) + brr.setUsepoms(true) + brr.setRoot(repo) + brr.setName(s"repo-${i + 1}") + cr.add(brr) + printStream.println(s"$repo added as a remote repository with the name: ${brr.getName}") + } + } + cr + } + + /** + * Output a comma-delimited list of paths for the downloaded jars to be added to the classpath + * (will append to jars in SparkSubmit). The name of the jar is given + * after a '!' by Ivy. It also sometimes contains '(bundle)' after '.jar'. Remove that as well. + * @param artifacts Sequence of dependencies that were resolved and retrieved + * @param cacheDirectory directory where jars are cached + * @return a comma-delimited list of paths for the dependencies + */ + private[spark] def resolveDependencyPaths( + artifacts: Array[AnyRef], + cacheDirectory: File): String = { + artifacts.map { artifactInfo => + val artifactString = artifactInfo.toString + val jarName = artifactString.drop(artifactString.lastIndexOf("!") + 1) + cacheDirectory.getAbsolutePath + File.separator + + jarName.substring(0, jarName.lastIndexOf(".jar") + 4) + }.mkString(",") + } + + /** Adds the given maven coordinates to Ivy's module descriptor. */ + private[spark] def addDependenciesToIvy( + md: DefaultModuleDescriptor, + artifacts: Seq[MavenCoordinate], + ivyConfName: String): Unit = { + artifacts.foreach { mvn => + val ri = ModuleRevisionId.newInstance(mvn.groupId, mvn.artifactId, mvn.version) + val dd = new DefaultDependencyDescriptor(ri, false, false) + dd.addDependencyConfiguration(ivyConfName, ivyConfName) + printStream.println(s"${dd.getDependencyId} added as a dependency") + md.addDependency(dd) + } + } + + /** A nice function to use in tests as well. Values are dummy strings. */ + private[spark] def getModuleDescriptor = DefaultModuleDescriptor.newDefaultInstance( + ModuleRevisionId.newInstance("org.apache.spark", "spark-submit-parent", "1.0")) + + /** + * Resolves any dependencies that were supplied through maven coordinates + * @param coordinates Comma-delimited string of maven coordinates + * @param remoteRepos Comma-delimited string of remote repositories other than maven central + * @param ivyPath The path to the local ivy repository + * @return The comma-delimited path to the jars of the given maven artifacts including their + * transitive dependencies + */ + private[spark] def resolveMavenCoordinates( + coordinates: String, + remoteRepos: Option[String], + ivyPath: Option[String], + isTest: Boolean = false): String = { + if (coordinates == null || coordinates.trim.isEmpty) { + "" + } else { + val artifacts = extractMavenCoordinates(coordinates) + // Default configuration name for ivy + val ivyConfName = "default" + // set ivy settings for location of cache + val ivySettings: IvySettings = new IvySettings + // Directories for caching downloads through ivy and storing the jars when maven coordinates + // are supplied to spark-submit + val alternateIvyCache = ivyPath.getOrElse("") + val packagesDirectory: File = + if (alternateIvyCache.trim.isEmpty) { + new File(ivySettings.getDefaultIvyUserDir, "jars") + } else { + ivySettings.setDefaultCache(new File(alternateIvyCache, "cache")) + new File(alternateIvyCache, "jars") + } + printStream.println( + s"Ivy Default Cache set to: ${ivySettings.getDefaultCache.getAbsolutePath}") + printStream.println(s"The jars for the packages stored in: $packagesDirectory") + // create a pattern matcher + ivySettings.addMatcher(new GlobPatternMatcher) + // create the dependency resolvers + val repoResolver = createRepoResolvers(remoteRepos) + ivySettings.addResolver(repoResolver) + ivySettings.setDefaultResolver(repoResolver.getName) + + val ivy = Ivy.newInstance(ivySettings) + // Set resolve options to download transitive dependencies as well + val resolveOptions = new ResolveOptions + resolveOptions.setTransitive(true) + val retrieveOptions = new RetrieveOptions + // Turn downloading and logging off for testing + if (isTest) { + resolveOptions.setDownload(false) + resolveOptions.setLog(LogOptions.LOG_QUIET) + retrieveOptions.setLog(LogOptions.LOG_QUIET) + } else { + resolveOptions.setDownload(true) + } + + // A Module descriptor must be specified. Entries are dummy strings + val md = getModuleDescriptor + md.setDefaultConf(ivyConfName) + + // Add an exclusion rule for Spark + val sparkArtifacts = new ArtifactId(new ModuleId("org.apache.spark", "*"), "*", "*", "*") + val sparkDependencyExcludeRule = + new DefaultExcludeRule(sparkArtifacts, ivySettings.getMatcher("glob"), null) + sparkDependencyExcludeRule.addConfiguration(ivyConfName) + + // Exclude any Spark dependencies, and add all supplied maven artifacts as dependencies + md.addExcludeRule(sparkDependencyExcludeRule) + addDependenciesToIvy(md, artifacts, ivyConfName) + + // resolve dependencies + val rr: ResolveReport = ivy.resolve(md, resolveOptions) + if (rr.hasError) { + throw new RuntimeException(rr.getAllProblemMessages.toString) + } + // retrieve all resolved dependencies + ivy.retrieve(rr.getModuleDescriptor.getModuleRevisionId, + packagesDirectory.getAbsolutePath + File.separator + "[artifact](-[classifier]).[ext]", + retrieveOptions.setConfs(Array(ivyConfName))) + + resolveDependencyPaths(rr.getArtifacts.toArray, packagesDirectory) + } + } +} + /** * Provides an indirection layer for passing arguments as system properties or flags to * the user's driver program or to downstream launcher tools. 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 73e921fd83ef2..5cadc534f4baa 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -50,6 +50,9 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St var name: String = null var childArgs: ArrayBuffer[String] = new ArrayBuffer[String]() var jars: String = null + var packages: String = null + var repositories: String = null + var ivyRepoPath: String = null var verbose: Boolean = false var isPython: Boolean = false var pyFiles: String = null @@ -123,6 +126,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St .orNull name = Option(name).orElse(sparkProperties.get("spark.app.name")).orNull jars = Option(jars).orElse(sparkProperties.get("spark.jars")).orNull + ivyRepoPath = sparkProperties.get("spark.jars.ivy").orNull deployMode = Option(deployMode).orElse(env.get("DEPLOY_MODE")).orNull numExecutors = Option(numExecutors) .getOrElse(sparkProperties.get("spark.executor.instances").orNull) @@ -212,6 +216,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St | name $name | childArgs [${childArgs.mkString(" ")}] | jars $jars + | packages $packages + | repositories $repositories | verbose $verbose | |Spark properties used, including those specified through @@ -318,6 +324,14 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St jars = Utils.resolveURIs(value) parse(tail) + case ("--packages") :: value :: tail => + packages = value + parse(tail) + + case ("--repositories") :: value :: tail => + repositories = value + parse(tail) + case ("--conf" | "-c") :: value :: tail => value.split("=", 2).toSeq match { case Seq(k, v) => sparkProperties(k) = v @@ -368,6 +382,13 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St | --name NAME A name of your application. | --jars JARS Comma-separated list of local jars to include on the driver | and executor classpaths. + | --packages Comma-separated list of maven coordinates of jars to include + | on the driver and executor classpaths. Will search the local + | maven repo, then maven central and any additional remote + | repositories given by --repositories. The format for the + | coordinates should be groupId:artifactId:version. + | --repositories Comma-separated list of additional remote repositories to + | search for the maven coordinates given with --packages. | --py-files PY_FILES Comma-separated list of .zip, .egg, or .py files to place | on the PYTHONPATH for Python apps. | --files FILES Comma-separated list of files to be placed in the working 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 82628ad3abd99..3f1355f82893e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -307,7 +307,21 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "--name", "testApp", "--master", "local-cluster[2,1,512]", "--jars", jarsString, - unusedJar.toString) + unusedJar.toString, "SparkSubmitClassA", "SparkSubmitClassB") + runSparkSubmit(args) + } + + test("includes jars passed in through --packages") { + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val packagesString = "com.databricks:spark-csv_2.10:0.1,com.databricks:spark-avro_2.10:0.1" + val args = Seq( + "--class", JarCreationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local-cluster[2,1,512]", + "--packages", packagesString, + "--conf", "spark.ui.enabled=false", + unusedJar.toString, + "com.databricks.spark.csv.DefaultSource", "com.databricks.spark.avro.DefaultSource") runSparkSubmit(args) } @@ -467,8 +481,8 @@ object JarCreationTest extends Logging { val result = sc.makeRDD(1 to 100, 10).mapPartitions { x => var exception: String = null try { - Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader) - Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader) + Class.forName(args(0), true, Thread.currentThread().getContextClassLoader) + Class.forName(args(1), true, Thread.currentThread().getContextClassLoader) } catch { case t: Throwable => exception = t + "\n" + t.getStackTraceString diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala new file mode 100644 index 0000000000000..53665350013cd --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy + +import java.io.{PrintStream, OutputStream, File} + +import scala.collection.mutable.ArrayBuffer + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.ivy.core.module.descriptor.MDArtifact +import org.apache.ivy.plugins.resolver.IBiblioResolver + +class SparkSubmitUtilsSuite extends FunSuite with BeforeAndAfterAll { + + private val noOpOutputStream = new OutputStream { + def write(b: Int) = {} + } + + /** Simple PrintStream that reads data into a buffer */ + private class BufferPrintStream extends PrintStream(noOpOutputStream) { + var lineBuffer = ArrayBuffer[String]() + override def println(line: String) { + lineBuffer += line + } + } + + override def beforeAll() { + super.beforeAll() + // We don't want to write logs during testing + SparkSubmitUtils.printStream = new BufferPrintStream + } + + test("incorrect maven coordinate throws error") { + val coordinates = Seq("a:b: ", " :a:b", "a: :b", "a:b:", ":a:b", "a::b", "::", "a:b", "a") + for (coordinate <- coordinates) { + intercept[IllegalArgumentException] { + SparkSubmitUtils.extractMavenCoordinates(coordinate) + } + } + } + + test("create repo resolvers") { + val resolver1 = SparkSubmitUtils.createRepoResolvers(None) + // should have central by default + assert(resolver1.getResolvers.size() === 1) + assert(resolver1.getResolvers.get(0).asInstanceOf[IBiblioResolver].getName === "central") + + val repos = "a/1,b/2,c/3" + val resolver2 = SparkSubmitUtils.createRepoResolvers(Option(repos)) + assert(resolver2.getResolvers.size() === 4) + 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 { + assert(resolver.getName === s"repo-$i") + assert(resolver.getRoot === expected(i - 1)) + } + } + } + + test("add dependencies works correctly") { + val md = SparkSubmitUtils.getModuleDescriptor + val artifacts = SparkSubmitUtils.extractMavenCoordinates("com.databricks:spark-csv_2.10:0.1," + + "com.databricks:spark-avro_2.10:0.1") + + SparkSubmitUtils.addDependenciesToIvy(md, artifacts, "default") + assert(md.getDependencies.length === 2) + } + + test("ivy path works correctly") { + val ivyPath = "dummy/ivy" + val md = SparkSubmitUtils.getModuleDescriptor + val artifacts = for (i <- 0 until 3) yield new MDArtifact(md, s"jar-$i", "jar", "jar") + var jPaths = SparkSubmitUtils.resolveDependencyPaths(artifacts.toArray, new File(ivyPath)) + for (i <- 0 until 3) { + val index = jPaths.indexOf(ivyPath) + assert(index >= 0) + jPaths = jPaths.substring(index + ivyPath.length) + } + // end to end + val jarPath = SparkSubmitUtils.resolveMavenCoordinates( + "com.databricks:spark-csv_2.10:0.1", None, Option(ivyPath), true) + assert(jarPath.indexOf(ivyPath) >= 0, "should use non-default ivy path") + } + + test("search for artifact at other repositories") { + val path = SparkSubmitUtils.resolveMavenCoordinates("com.agimatec:agimatec-validation:0.9.3", + Option("https://oss.sonatype.org/content/repositories/agimatec/"), None, true) + assert(path.indexOf("agimatec-validation") >= 0, "should find package. If it doesn't, check" + + "if package still exists. If it has been removed, replace the example in this test.") + } + + test("dependency not found throws RuntimeException") { + intercept[RuntimeException] { + SparkSubmitUtils.resolveMavenCoordinates("a:b:c", None, None, true) + } + } + + test("neglects Spark and Spark's dependencies") { + val path = SparkSubmitUtils.resolveMavenCoordinates( + "org.apache.spark:spark-core_2.10:1.2.0", None, None, true) + assert(path === "", "should return empty path") + } +} diff --git a/pom.xml b/pom.xml index 542efbaf06eb0..55d82ad298def 100644 --- a/pom.xml +++ b/pom.xml @@ -136,6 +136,8 @@ 1.2.3 8.1.14.v20131031 0.5.0 + 2.4.0 + 2.0.8 3.1.0 1.7.6 From 4cf4cba08f1757ec0d9bffdfae6db719a4fb5a3f Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 4 Feb 2015 00:40:28 -0800 Subject: [PATCH 060/578] [SPARK-5379][Streaming] Add awaitTerminationOrTimeout Added `awaitTerminationOrTimeout` to return if the waiting time elapsed: * `true` if it's stopped. * `false` if the waiting time elapsed before returning from the method. * throw the reported error if it's thrown during the execution. Also deprecated `awaitTermination(timeout: Long)`. Author: zsxwing Closes #4171 from zsxwing/SPARK-5379 and squashes the following commits: c9e660b [zsxwing] Add a unit test for awaitTerminationOrTimeout 8a89f92 [zsxwing] Add awaitTerminationOrTimeout to python cdc820b [zsxwing] Add awaitTerminationOrTimeout --- python/pyspark/streaming/context.py | 9 +++++++ .../spark/streaming/StreamingContext.scala | 13 ++++++++++ .../api/java/JavaStreamingContext.scala | 13 ++++++++++ .../streaming/StreamingContextSuite.scala | 24 +++++++++++++++++++ 4 files changed, 59 insertions(+) diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py index 18aaae93b05f2..b06ab650370bd 100644 --- a/python/pyspark/streaming/context.py +++ b/python/pyspark/streaming/context.py @@ -191,6 +191,15 @@ def awaitTermination(self, timeout=None): else: self._jssc.awaitTermination(int(timeout * 1000)) + def awaitTerminationOrTimeout(self, timeout): + """ + Wait for the execution to stop. Return `true` if it's stopped; or + throw the reported error during the execution; or `false` if the + waiting time elapsed before returning from the method. + @param timeout: time to wait in seconds + """ + self._jssc.awaitTerminationOrTimeout(int(timeout * 1000)) + def stop(self, stopSparkContext=True, stopGraceFully=False): """ Stop the execution of the streams, with option of ensuring all 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 ddc435cf1a2e6..ba3f23434f24c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -526,10 +526,23 @@ class StreamingContext private[streaming] ( * will be thrown in this thread. * @param timeout time to wait in milliseconds */ + @deprecated("Use awaitTerminationOrTimeout(Long) instead", "1.3.0") def awaitTermination(timeout: Long) { waiter.waitForStopOrError(timeout) } + /** + * Wait for the execution to stop. Any exceptions that occurs during the execution + * will be thrown in this thread. + * + * @param timeout time to wait in milliseconds + * @return `true` if it's stopped; or throw the reported error during the execution; or `false` + * if the waiting time elapsed before returning from the method. + */ + def awaitTerminationOrTimeout(timeout: Long): Boolean = { + waiter.waitForStopOrError(timeout) + } + /** * Stop the execution of the streams immediately (does not wait for all received data * to be processed). 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 0f7ae7a1c7de8..e3db01c1e12c6 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 @@ -597,10 +597,23 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { * will be thrown in this thread. * @param timeout time to wait in milliseconds */ + @deprecated("Use awaitTerminationOrTimeout(Long) instead", "1.3.0") def awaitTermination(timeout: Long): Unit = { ssc.awaitTermination(timeout) } + /** + * Wait for the execution to stop. Any exceptions that occurs during the execution + * will be thrown in this thread. + * + * @param timeout time to wait in milliseconds + * @return `true` if it's stopped; or throw the reported error during the execution; or `false` + * if the waiting time elapsed before returning from the method. + */ + def awaitTerminationOrTimeout(timeout: Long): Boolean = { + ssc.awaitTerminationOrTimeout(timeout) + } + /** * Stop the execution of the streams. Will stop the associated JavaSparkContext as well. */ 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 0b5af25e0f7cc..2aa5e0876b6e0 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -304,6 +304,30 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w assert(exception.getMessage.contains("transform"), "Expected exception not thrown") } + test("awaitTerminationOrTimeout") { + ssc = new StreamingContext(master, appName, batchDuration) + val inputStream = addInputStream(ssc) + inputStream.map(x => x).register() + + ssc.start() + + // test whether awaitTerminationOrTimeout() return false after give amount of time + failAfter(1000 millis) { + assert(ssc.awaitTerminationOrTimeout(500) === false) + } + + // test whether awaitTerminationOrTimeout() return true if context is stopped + failAfter(10000 millis) { // 10 seconds because spark takes a long time to shutdown + new Thread() { + override def run() { + Thread.sleep(500) + ssc.stop() + } + }.start() + assert(ssc.awaitTerminationOrTimeout(10000) === true) + } + } + test("DStream and generated RDD creation sites") { testPackage.test() } From a74cbbf12fa59df37eb7172652138c78707d33d8 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 4 Feb 2015 00:52:41 -0800 Subject: [PATCH 061/578] [Minor] Fix incorrect warning log The warning log looks incorrect. Just fix it. Author: Liang-Chi Hsieh Closes #4360 from viirya/fixing_typo and squashes the following commits: 48fbe4f [Liang-Chi Hsieh] Fix incorrect warning log. --- .../org/apache/spark/streaming/scheduler/ReceiverTracker.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 00456ab2a0c92..b36aeb341d25e 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 @@ -254,7 +254,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false // Check if all the receivers have been deregistered or not if (!receiverInfo.isEmpty) { - logWarning("All of the receivers have not deregistered, " + receiverInfo) + logWarning("Not all of the receivers have deregistered, " + receiverInfo) } else { logInfo("All of the receivers have deregistered successfully") } From 5aa0f219d1f8e8d3bb083dd547532ebb72ba7a15 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 4 Feb 2015 01:02:20 -0800 Subject: [PATCH 062/578] [SPARK-5574] use given name prefix in dir https://issues.apache.org/jira/browse/SPARK-5574 very minor, doesn't effect external behavior at all. Note that after this change, some of these dirs no longer will have "spark" in the name at all. I could change those locations that do pass in a name prefix to also include "spark", eg. "blockmgr" -> "spark-blockmgr" Author: Imran Rashid Closes #4344 from squito/SPARK-5574 and squashes the following commits: 33a84fe [Imran Rashid] use given name prefix in dir --- 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 e9f2aed9ffbea..3c8a0e40bf785 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -279,7 +279,7 @@ private[spark] object Utils extends Logging { maxAttempts + " attempts!") } try { - dir = new File(root, "spark-" + UUID.randomUUID.toString) + dir = new File(root, namePrefix + "-" + UUID.randomUUID.toString) if (dir.exists() || !dir.mkdirs()) { dir = null } else { From 38a416f0360fa68fc445af14910fb253ff9ad493 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 4 Feb 2015 08:54:20 -0800 Subject: [PATCH 063/578] [SPARK-5585] Flaky test in MLlib python Add a seed for tests. Author: Davies Liu Closes #4358 from davies/flaky_test and squashes the following commits: 02371c3 [Davies Liu] Merge branch 'master' of github.com:apache/spark into flaky_test ced499b [Davies Liu] add seed for test --- python/pyspark/mllib/tests.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 42aa22873772d..49e5c9d58e5db 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -285,7 +285,7 @@ def test_regression(self): self.assertTrue(dt_model.predict(features[3]) > 0) rf_model = RandomForest.trainRegressor( - rdd, categoricalFeaturesInfo=categoricalFeaturesInfo, numTrees=100) + rdd, categoricalFeaturesInfo=categoricalFeaturesInfo, numTrees=100, seed=1) self.assertTrue(rf_model.predict(features[0]) <= 0) self.assertTrue(rf_model.predict(features[1]) > 0) self.assertTrue(rf_model.predict(features[2]) <= 0) From ac0b2b788ff144970d6fdbdc445367772770458d Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 4 Feb 2015 11:34:46 -0800 Subject: [PATCH 064/578] [SPARK-5588] [SQL] support select/filter by SQL expression ``` df.selectExpr('a + 1', 'abs(age)') df.filter('age > 3') df[ df.age > 3 ] df[ ['age', 'name'] ] ``` Author: Davies Liu Closes #4359 from davies/select_expr and squashes the following commits: d99856b [Davies Liu] support select/filter by SQL expression --- .../apache/spark/api/python/PythonUtils.scala | 11 +++- python/pyspark/sql.py | 53 +++++++++++++++---- .../main/scala/org/apache/spark/sql/Dsl.scala | 11 ---- 3 files changed, 53 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index b7cfc8bd9c542..acbaba6791850 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -17,8 +17,10 @@ package org.apache.spark.api.python -import java.io.{File, InputStream, IOException, OutputStream} +import java.io.{File} +import java.util.{List => JList} +import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkContext @@ -44,4 +46,11 @@ private[spark] object PythonUtils { def generateRDDWithNull(sc: JavaSparkContext): JavaRDD[String] = { sc.parallelize(List("a", null, "b")) } + + /** + * Convert list of T into seq of T (for calling API with varargs) + */ + def toSeq[T](cols: JList[T]): Seq[T] = { + cols.toList.toSeq + } } diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 74305dea749c8..a266cde51d317 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -2128,7 +2128,7 @@ def sort(self, *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.Dsl.toColumns(jcols)) + jdf = self._jdf.sort(self._sc._jvm.PythonUtils.toSeq(jcols)) return DataFrame(jdf, self.sql_ctx) sortBy = sort @@ -2159,13 +2159,20 @@ def __getitem__(self, item): >>> 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) - - # TODO projection - raise IndexError + 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 @@ -2194,18 +2201,44 @@ def select(self, *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.Dsl.toColumns(jcols)) + 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. + """ 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')] """ - return DataFrame(self._jdf.filter(condition._jc), self.sql_ctx) + 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 @@ -2223,7 +2256,7 @@ def groupBy(self, *cols): """ 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.Dsl.toColumns(jcols)) + jdf = self._jdf.groupBy(self.sql_ctx._sc._jvm.PythonUtils.toSeq(jcols)) return GroupedDataFrame(jdf, self.sql_ctx) def agg(self, *exprs): @@ -2338,7 +2371,7 @@ def agg(self, *exprs): 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.Dsl.toColumns(jcols)) + jdf = self._jdf.agg(exprs[0]._jc, self.sql_ctx._sc._jvm.PythonUtils.toSeq(jcols)) return DataFrame(jdf, self.sql_ctx) @dfapi @@ -2633,7 +2666,7 @@ def countDistinct(col, *cols): 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.Dsl.toColumns(jcols)) + sc._jvm.PythonUtils.toSeq(jcols)) return Column(jc) @staticmethod 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 8cf59f0a1f099..50f442dd87bf3 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 @@ -17,11 +17,8 @@ package org.apache.spark.sql -import java.util.{List => JList} - import scala.language.implicitConversions import scala.reflect.runtime.universe.{TypeTag, typeTag} -import scala.collection.JavaConversions._ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions._ @@ -169,14 +166,6 @@ object Dsl { /** Computes the absolutle value. */ def abs(e: Column): Column = Abs(e.expr) - /** - * This is a private API for Python - * TODO: move this to a private package - */ - def toColumns(cols: JList[Column]): Seq[Column] = { - cols.toList.toSeq - } - ////////////////////////////////////////////////////////////////////////////////////////////// ////////////////////////////////////////////////////////////////////////////////////////////// From b0c0021953826bccaee818a54afc44e8bdfa8572 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 4 Feb 2015 12:06:34 -0800 Subject: [PATCH 065/578] [SPARK-4964] [Streaming] Exactly-once semantics for Kafka Author: cody koeninger Closes #3798 from koeninger/kafkaRdd and squashes the following commits: 1dc2941 [cody koeninger] [SPARK-4964] silence ConsumerConfig warnings about broker connection props 59e29f6 [cody koeninger] [SPARK-4964] settle on "Direct" as a naming convention for the new stream 8c31855 [cody koeninger] [SPARK-4964] remove HasOffsetRanges interface from return types 0df3ebe [cody koeninger] [SPARK-4964] add comments per pwendell / dibbhatt 8991017 [cody koeninger] [SPARK-4964] formatting 825110f [cody koeninger] [SPARK-4964] rename stuff per TD 4354bce [cody koeninger] [SPARK-4964] per td, remove java interfaces, replace with final classes, corresponding changes to KafkaRDD constructor and checkpointing 9adaa0a [cody koeninger] [SPARK-4964] formatting 0090553 [cody koeninger] [SPARK-4964] javafication of interfaces 9a838c2 [cody koeninger] [SPARK-4964] code cleanup, add more tests 2b340d8 [cody koeninger] [SPARK-4964] refactor per TD feedback 80fd6ae [cody koeninger] [SPARK-4964] Rename createExactlyOnceStream so it isnt over-promising, change doc 99d2eba [cody koeninger] [SPARK-4964] Reduce level of nesting. If beginning is past end, its actually an error (may happen if Kafka topic was deleted and recreated) 19406cc [cody koeninger] Merge branch 'master' of https://github.com/apache/spark into kafkaRdd 2e67117 [cody koeninger] [SPARK-4964] one potential way of hiding most of the implementation, while still allowing access to offsets (but not subclassing) bb80bbe [cody koeninger] [SPARK-4964] scalastyle line length d4a7cf7 [cody koeninger] [SPARK-4964] allow for use cases that need to override compute for custom kafka dstreams c1bd6d9 [cody koeninger] [SPARK-4964] use newly available attemptNumber for correct retry behavior 548d529 [cody koeninger] Merge branch 'master' of https://github.com/apache/spark into kafkaRdd 0458e4e [cody koeninger] [SPARK-4964] recovery of generated rdds from checkpoint e86317b [cody koeninger] [SPARK-4964] try seed brokers in random order to spread metadata requests e93eb72 [cody koeninger] [SPARK-4964] refactor to add preferredLocations. depends on SPARK-4014 356c7cc [cody koeninger] [SPARK-4964] code cleanup per helena adf99a6 [cody koeninger] [SPARK-4964] fix serialization issues for checkpointing 1d50749 [cody koeninger] [SPARK-4964] code cleanup per tdas 8bfd6c0 [cody koeninger] [SPARK-4964] configure rate limiting via spark.streaming.receiver.maxRate e09045b [cody koeninger] [SPARK-4964] add foreachPartitionWithIndex, to avoid doing equivalent map + empty foreach boilerplate cac63ee [cody koeninger] additional testing, fix fencepost error 37d3053 [cody koeninger] make KafkaRDDPartition available to users so offsets can be committed per partition bcca8a4 [cody koeninger] Merge branch 'master' of https://github.com/apache/spark into kafkaRdd 6bf14f2 [cody koeninger] first attempt at a Kafka dstream that allows for exactly-once semantics 326ff3c [cody koeninger] add some tests 38bb727 [cody koeninger] give easy access to the parameters of a KafkaRDD 979da25 [cody koeninger] dont allow empty leader offsets to be returned 8d7de4a [cody koeninger] make sure leader offsets can be found even for leaders that arent in the seed brokers 4b078bf [cody koeninger] differentiate between leader and consumer offsets in error message 3c2a96a [cody koeninger] fix scalastyle errors 29c6b43 [cody koeninger] cleanup logging 783b477 [cody koeninger] update tests for kafka 8.1.1 7d050bc [cody koeninger] methods to set consumer offsets and get topic metadata, switch back to inclusive start / exclusive end to match typical kafka consumer behavior ce91c59 [cody koeninger] method to get consumer offsets, explicit error handling 4dafd1b [cody koeninger] method to get leader offsets, switch rdd bound to being exclusive start, inclusive end to match offsets typically returned from cluster 0b94b33 [cody koeninger] use dropWhile rather than filter to trim beginning of fetch response 1d70625 [cody koeninger] WIP on kafka cluster 76913e2 [cody koeninger] Batch oriented kafka rdd, WIP. todo: cluster metadata / finding leader --- external/kafka/pom.xml | 2 +- .../kafka/DirectKafkaInputDStream.scala | 162 ++++++++ .../spark/streaming/kafka/KafkaCluster.scala | 369 ++++++++++++++++++ .../spark/streaming/kafka/KafkaRDD.scala | 224 +++++++++++ .../streaming/kafka/KafkaRDDPartition.scala | 59 +++ .../spark/streaming/kafka/KafkaUtils.scala | 178 ++++++++- .../apache/spark/streaming/kafka/Leader.scala | 46 +++ .../spark/streaming/kafka/OffsetRange.scala | 70 ++++ .../streaming/kafka/KafkaClusterSuite.scala | 73 ++++ .../kafka/KafkaDirectStreamSuite.scala | 92 +++++ .../spark/streaming/kafka/KafkaRDDSuite.scala | 99 +++++ .../streaming/kafka/KafkaStreamSuite.scala | 8 +- 12 files changed, 1376 insertions(+), 6 deletions(-) create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Leader.scala create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala create mode 100644 external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala create mode 100644 external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaDirectStreamSuite.scala create mode 100644 external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index b29b0509656ba..af96138d79405 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -44,7 +44,7 @@ org.apache.kafka kafka_${scala.binary.version} - 0.8.0 + 0.8.1.1 com.sun.jmx 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 new file mode 100644 index 0000000000000..c7bca43eb889d --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.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.streaming.kafka + + +import scala.annotation.tailrec +import scala.collection.mutable +import scala.reflect.{classTag, ClassTag} + +import kafka.common.TopicAndPartition +import kafka.message.MessageAndMetadata +import kafka.serializer.Decoder + +import org.apache.spark.{Logging, SparkException} +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset +import org.apache.spark.streaming.{StreamingContext, Time} +import org.apache.spark.streaming.dstream._ + +/** + * A stream of {@link org.apache.spark.streaming.kafka.KafkaRDD} where + * each given Kafka topic/partition corresponds to an RDD partition. + * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number + * of messages + * per second that each '''partition''' will accept. + * Starting offsets are specified in advance, + * and this DStream is not responsible for committing offsets, + * so that you can control exactly-once semantics. + * For an easy interface to Kafka-managed offsets, + * see {@link org.apache.spark.streaming.kafka.KafkaCluster} + * @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 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, + R: ClassTag]( + @transient ssc_ : StreamingContext, + val kafkaParams: Map[String, String], + val fromOffsets: Map[TopicAndPartition, Long], + messageHandler: MessageAndMetadata[K, V] => R +) extends InputDStream[R](ssc_) with Logging { + val maxRetries = context.sparkContext.getConf.getInt( + "spark.streaming.kafka.maxRetries", 1) + + protected[streaming] override val checkpointData = + new DirectKafkaInputDStreamCheckpointData + + protected val kc = new KafkaCluster(kafkaParams) + + protected val maxMessagesPerPartition: Option[Long] = { + val ratePerSec = context.sparkContext.getConf.getInt( + "spark.streaming.kafka.maxRatePerPartition", 0) + if (ratePerSec > 0) { + val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000 + Some((secsPerBatch * ratePerSec).toLong) + } else { + None + } + } + + protected var currentOffsets = fromOffsets + + @tailrec + protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, LeaderOffset] = { + val o = kc.getLatestLeaderOffsets(currentOffsets.keySet) + // Either.fold would confuse @tailrec, do it manually + if (o.isLeft) { + val err = o.left.get.toString + if (retries <= 0) { + throw new SparkException(err) + } else { + log.error(err) + Thread.sleep(kc.config.refreshLeaderBackoffMs) + latestLeaderOffsets(retries - 1) + } + } else { + o.right.get + } + } + + // limits the maximum number of messages per partition + protected def clamp( + leaderOffsets: Map[TopicAndPartition, LeaderOffset]): Map[TopicAndPartition, LeaderOffset] = { + maxMessagesPerPartition.map { mmp => + leaderOffsets.map { case (tp, lo) => + tp -> lo.copy(offset = Math.min(currentOffsets(tp) + mmp, lo.offset)) + } + }.getOrElse(leaderOffsets) + } + + override def compute(validTime: Time): Option[KafkaRDD[K, V, U, T, R]] = { + val untilOffsets = clamp(latestLeaderOffsets(maxRetries)) + val rdd = KafkaRDD[K, V, U, T, R]( + context.sparkContext, kafkaParams, currentOffsets, untilOffsets, messageHandler) + + currentOffsets = untilOffsets.map(kv => kv._1 -> kv._2.offset) + Some(rdd) + } + + override def start(): Unit = { + } + + def stop(): Unit = { + } + + private[streaming] + class DirectKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) { + def batchForTime = data.asInstanceOf[mutable.HashMap[ + Time, Array[OffsetRange.OffsetRangeTuple]]] + + override def update(time: Time) { + batchForTime.clear() + generatedRDDs.foreach { kv => + val a = kv._2.asInstanceOf[KafkaRDD[K, V, U, T, R]].offsetRanges.map(_.toTuple).toArray + batchForTime += kv._1 -> a + } + } + + override def cleanup(time: Time) { } + + override def restore() { + // this is assuming that the topics don't change during execution, which is true currently + val topics = fromOffsets.keySet + val leaders = kc.findLeaders(topics).fold( + errs => throw new SparkException(errs.mkString("\n")), + ok => ok + ) + + batchForTime.toSeq.sortBy(_._1)(Time.ordering).foreach { case (t, b) => + logInfo(s"Restoring KafkaRDD for time $t ${b.mkString("[", ", ", "]")}") + generatedRDDs += t -> new KafkaRDD[K, V, U, T, R]( + context.sparkContext, kafkaParams, b.map(OffsetRange(_)), leaders, messageHandler) + } + } + } + +} 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 new file mode 100644 index 0000000000000..ccc62bfe8f057 --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala @@ -0,0 +1,369 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.control.NonFatal +import scala.util.Random +import scala.collection.mutable.ArrayBuffer +import java.util.Properties +import kafka.api._ +import kafka.common.{ErrorMapping, OffsetMetadataAndError, TopicAndPartition} +import kafka.consumer.{ConsumerConfig, SimpleConsumer} +import org.apache.spark.SparkException + +/** + * Convenience methods for interacting with a Kafka cluster. + * @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 + */ +private[spark] +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { + import KafkaCluster.{Err, LeaderOffset, SimpleConsumerConfig} + + // ConsumerConfig isn't serializable + @transient private var _config: SimpleConsumerConfig = null + + def config: SimpleConsumerConfig = this.synchronized { + if (_config == null) { + _config = SimpleConsumerConfig(kafkaParams) + } + _config + } + + def connect(host: String, port: Int): SimpleConsumer = + new SimpleConsumer(host, port, config.socketTimeoutMs, + config.socketReceiveBufferBytes, config.clientId) + + def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] = + findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2)) + + // Metadata api + // scalastyle:off + // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI + // scalastyle:on + + def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = { + val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion, + 0, config.clientId, Seq(topic)) + val errs = new Err + withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => + val resp: TopicMetadataResponse = consumer.send(req) + resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata => + tm.partitionsMetadata.find(_.partitionId == partition) + }.foreach { pm: PartitionMetadata => + pm.leader.foreach { leader => + return Right((leader.host, leader.port)) + } + } + } + Left(errs) + } + + def findLeaders( + topicAndPartitions: Set[TopicAndPartition] + ): Either[Err, Map[TopicAndPartition, (String, Int)]] = { + val topics = topicAndPartitions.map(_.topic) + val response = getPartitionMetadata(topics).right + val answer = response.flatMap { tms: Set[TopicMetadata] => + val leaderMap = tms.flatMap { tm: TopicMetadata => + tm.partitionsMetadata.flatMap { pm: PartitionMetadata => + val tp = TopicAndPartition(tm.topic, pm.partitionId) + if (topicAndPartitions(tp)) { + pm.leader.map { l => + tp -> (l.host -> l.port) + } + } else { + None + } + } + }.toMap + + if (leaderMap.keys.size == topicAndPartitions.size) { + Right(leaderMap) + } else { + val missing = topicAndPartitions.diff(leaderMap.keySet) + val err = new Err + err.append(new SparkException(s"Couldn't find leaders for ${missing}")) + Left(err) + } + } + answer + } + + def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] = { + getPartitionMetadata(topics).right.map { r => + r.flatMap { tm: TopicMetadata => + tm.partitionsMetadata.map { pm: PartitionMetadata => + TopicAndPartition(tm.topic, pm.partitionId) + } + } + } + } + + def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = { + val req = TopicMetadataRequest( + TopicMetadataRequest.CurrentVersion, 0, config.clientId, topics.toSeq) + 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) + } + Left(errs) + } + + // Leader offset api + // scalastyle:off + // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI + // scalastyle:on + + def getLatestLeaderOffsets( + topicAndPartitions: Set[TopicAndPartition] + ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = + getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime) + + def getEarliestLeaderOffsets( + topicAndPartitions: Set[TopicAndPartition] + ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = + getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime) + + def getLeaderOffsets( + topicAndPartitions: Set[TopicAndPartition], + before: Long + ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = { + getLeaderOffsets(topicAndPartitions, before, 1).right.map { r => + r.map { kv => + // mapValues isnt serializable, see SI-7005 + kv._1 -> kv._2.head + } + } + } + + private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] = + m.groupBy(_._2).map { kv => + kv._1 -> kv._2.keys.toSeq + } + + def getLeaderOffsets( + topicAndPartitions: Set[TopicAndPartition], + before: Long, + maxNumOffsets: Int + ): Either[Err, Map[TopicAndPartition, Seq[LeaderOffset]]] = { + findLeaders(topicAndPartitions).right.flatMap { tpToLeader => + val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader) + val leaders = leaderToTp.keys + var result = Map[TopicAndPartition, Seq[LeaderOffset]]() + val errs = new Err + withBrokers(leaders, errs) { consumer => + val partitionsToGetOffsets: Seq[TopicAndPartition] = + leaderToTp((consumer.host, consumer.port)) + val reqMap = partitionsToGetOffsets.map { tp: TopicAndPartition => + tp -> PartitionOffsetRequestInfo(before, maxNumOffsets) + }.toMap + val req = OffsetRequest(reqMap) + val resp = consumer.getOffsetsBefore(req) + val respMap = resp.partitionErrorAndOffsets + partitionsToGetOffsets.foreach { tp: TopicAndPartition => + respMap.get(tp).foreach { por: PartitionOffsetsResponse => + if (por.error == ErrorMapping.NoError) { + if (por.offsets.nonEmpty) { + result += tp -> por.offsets.map { off => + LeaderOffset(consumer.host, consumer.port, off) + } + } else { + errs.append(new SparkException( + s"Empty offsets for ${tp}, is ${before} before log beginning?")) + } + } else { + errs.append(ErrorMapping.exceptionFor(por.error)) + } + } + } + if (result.keys.size == topicAndPartitions.size) { + return Right(result) + } + } + val missing = topicAndPartitions.diff(result.keySet) + errs.append(new SparkException(s"Couldn't find leader offsets for ${missing}")) + Left(errs) + } + } + + // Consumer offset api + // scalastyle:off + // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI + // scalastyle:on + + /** Requires Kafka >= 0.8.1.1 */ + def getConsumerOffsets( + groupId: String, + topicAndPartitions: Set[TopicAndPartition] + ): Either[Err, Map[TopicAndPartition, Long]] = { + getConsumerOffsetMetadata(groupId, topicAndPartitions).right.map { r => + r.map { kv => + kv._1 -> kv._2.offset + } + } + } + + /** Requires Kafka >= 0.8.1.1 */ + def getConsumerOffsetMetadata( + groupId: String, + topicAndPartitions: Set[TopicAndPartition] + ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = { + var result = Map[TopicAndPartition, OffsetMetadataAndError]() + val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq) + val errs = new Err + withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => + val resp = consumer.fetchOffsets(req) + val respMap = resp.requestInfo + val needed = topicAndPartitions.diff(result.keySet) + needed.foreach { tp: TopicAndPartition => + respMap.get(tp).foreach { ome: OffsetMetadataAndError => + if (ome.error == ErrorMapping.NoError) { + result += tp -> ome + } else { + errs.append(ErrorMapping.exceptionFor(ome.error)) + } + } + } + if (result.keys.size == topicAndPartitions.size) { + return Right(result) + } + } + val missing = topicAndPartitions.diff(result.keySet) + errs.append(new SparkException(s"Couldn't find consumer offsets for ${missing}")) + Left(errs) + } + + /** Requires Kafka >= 0.8.1.1 */ + def setConsumerOffsets( + groupId: String, + offsets: Map[TopicAndPartition, Long] + ): Either[Err, Map[TopicAndPartition, Short]] = { + setConsumerOffsetMetadata(groupId, offsets.map { kv => + kv._1 -> OffsetMetadataAndError(kv._2) + }) + } + + /** Requires Kafka >= 0.8.1.1 */ + def setConsumerOffsetMetadata( + groupId: String, + metadata: Map[TopicAndPartition, OffsetMetadataAndError] + ): Either[Err, Map[TopicAndPartition, Short]] = { + var result = Map[TopicAndPartition, Short]() + val req = OffsetCommitRequest(groupId, metadata) + val errs = new Err + val topicAndPartitions = metadata.keySet + withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => + val resp = consumer.commitOffsets(req) + val respMap = resp.requestInfo + val needed = topicAndPartitions.diff(result.keySet) + needed.foreach { tp: TopicAndPartition => + respMap.get(tp).foreach { err: Short => + if (err == ErrorMapping.NoError) { + result += tp -> err + } else { + errs.append(ErrorMapping.exceptionFor(err)) + } + } + } + if (result.keys.size == topicAndPartitions.size) { + return Right(result) + } + } + val missing = topicAndPartitions.diff(result.keySet) + errs.append(new SparkException(s"Couldn't set offsets for ${missing}")) + Left(errs) + } + + // Try a call against potentially multiple brokers, accumulating errors + private def withBrokers(brokers: Iterable[(String, Int)], errs: Err) + (fn: SimpleConsumer => Any): Unit = { + brokers.foreach { hp => + var consumer: SimpleConsumer = null + try { + consumer = connect(hp._1, hp._2) + fn(consumer) + } catch { + case NonFatal(e) => + errs.append(e) + } finally { + if (consumer != null) { + consumer.close() + } + } + } + } +} + +private[spark] +object KafkaCluster { + type Err = ArrayBuffer[Throwable] + + private[spark] + case class LeaderOffset(host: String, port: Int, offset: Long) + + /** + * High-level kafka consumers connect to ZK. ConsumerConfig assumes this use case. + * Simple consumers connect directly to brokers, but need many of the same configs. + * This subclass won't warn about missing ZK params, or presence of broker params. + */ + private[spark] + class SimpleConsumerConfig private(brokers: String, originalProps: Properties) + extends ConsumerConfig(originalProps) { + val seedBrokers: Array[(String, Int)] = brokers.split(",").map { hp => + val hpa = hp.split(":") + (hpa(0), hpa(1).toInt) + } + } + + private[spark] + object SimpleConsumerConfig { + /** + * Make a consumer config without requiring group.id or zookeeper.connect, + * since communicating with brokers also needs common settings such as timeout + */ + def apply(kafkaParams: Map[String, String]): SimpleConsumerConfig = { + // These keys are from other pre-existing kafka configs for specifying brokers, accept either + val brokers = kafkaParams.get("metadata.broker.list") + .orElse(kafkaParams.get("bootstrap.servers")) + .getOrElse(throw new SparkException( + "Must specify metadata.broker.list or bootstrap.servers")) + + val props = new Properties() + kafkaParams.foreach { case (key, value) => + // prevent warnings on parameters ConsumerConfig doesn't know about + if (key != "metadata.broker.list" && key != "bootstrap.servers") { + props.put(key, value) + } + } + + Seq("zookeeper.connect", "group.id").foreach { s => + if (!props.contains(s)) { + props.setProperty(s, "") + } + } + + new SimpleConsumerConfig(brokers, props) + } + } +} 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 new file mode 100644 index 0000000000000..50bf7cbdb8dbf --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -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.streaming.kafka + +import scala.reflect.{classTag, ClassTag} + +import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskContext} +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.message.{MessageAndMetadata, MessageAndOffset} +import kafka.serializer.Decoder +import kafka.utils.VerifiableProperties + +/** + * A batch-oriented interface for consuming from Kafka. + * 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 + * @param messageHandler function for translating each message into the desired type + */ +private[spark] +class KafkaRDD[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag, + R: ClassTag] private[spark] ( + sc: SparkContext, + kafkaParams: Map[String, String], + val offsetRanges: Array[OffsetRange], + leaders: Map[TopicAndPartition, (String, Int)], + messageHandler: MessageAndMetadata[K, V] => R + ) extends RDD[R](sc, Nil) with Logging with HasOffsetRanges { + override def getPartitions: Array[Partition] = { + offsetRanges.zipWithIndex.map { case (o, i) => + val (host, port) = leaders(TopicAndPartition(o.topic, o.partition)) + new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port) + }.toArray + } + + override def getPreferredLocations(thePart: Partition): Seq[String] = { + val part = thePart.asInstanceOf[KafkaRDDPartition] + // TODO is additional hostname resolution necessary here + Seq(part.host) + } + + private def errBeginAfterEnd(part: KafkaRDDPartition): String = + s"Beginning offset ${part.fromOffset} is after the ending offset ${part.untilOffset} " + + s"for topic ${part.topic} partition ${part.partition}. " + + "You either provided an invalid fromOffset, or the Kafka topic has been damaged" + + private def errRanOutBeforeEnd(part: KafkaRDDPartition): String = + s"Ran out of messages before reaching ending offset ${part.untilOffset} " + + s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." + + " This should not happen, and indicates that messages may have been lost" + + private def errOvershotEnd(itemOffset: Long, part: KafkaRDDPartition): String = + s"Got ${itemOffset} > ending offset ${part.untilOffset} " + + s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." + + " This should not happen, and indicates a message may have been skipped" + + override def compute(thePart: Partition, context: TaskContext): Iterator[R] = { + 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 " + + s"skipping ${part.topic} ${part.partition}") + Iterator.empty + } else { + new KafkaRDDIterator(part, context) + } + } + + private class KafkaRDDIterator( + part: KafkaRDDPartition, + context: TaskContext) extends NextIterator[R] { + + context.addTaskCompletionListener{ context => closeIfNeeded() } + + log.info(s"Computing topic ${part.topic}, partition ${part.partition} " + + s"offsets ${part.fromOffset} -> ${part.untilOffset}") + + val kc = new KafkaCluster(kafkaParams) + val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) + .newInstance(kc.config.props) + .asInstanceOf[Decoder[K]] + val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) + .newInstance(kc.config.props) + .asInstanceOf[Decoder[V]] + val consumer = connectLeader + var requestOffset = part.fromOffset + var iter: Iterator[MessageAndOffset] = null + + // The idea is to use the provided preferred host, except on task retry atttempts, + // to minimize number of kafka metadata requests + private def connectLeader: SimpleConsumer = { + if (context.attemptNumber > 0) { + kc.connectLeader(part.topic, part.partition).fold( + errs => throw new SparkException( + s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " + + errs.mkString("\n")), + consumer => consumer + ) + } else { + kc.connect(part.host, part.port) + } + } + + private def handleFetchErr(resp: FetchResponse) { + if (resp.hasError) { + val err = resp.errorCode(part.topic, part.partition) + if (err == ErrorMapping.LeaderNotAvailableCode || + err == ErrorMapping.NotLeaderForPartitionCode) { + log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " + + s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms") + Thread.sleep(kc.config.refreshLeaderBackoffMs) + } + // Let normal rdd retry sort out reconnect attempts + throw ErrorMapping.exceptionFor(err) + } + } + + private def fetchBatch: Iterator[MessageAndOffset] = { + val req = new FetchRequestBuilder() + .addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes) + .build() + val resp = consumer.fetch(req) + handleFetchErr(resp) + // kafka may return a batch that starts before the requested offset + resp.messageSet(part.topic, part.partition) + .iterator + .dropWhile(_.offset < requestOffset) + } + + override def close() = consumer.close() + + override def getNext(): R = { + if (iter == null || !iter.hasNext) { + iter = fetchBatch + } + if (!iter.hasNext) { + assert(requestOffset == part.untilOffset, errRanOutBeforeEnd(part)) + finished = true + null.asInstanceOf[R] + } else { + val item = iter.next() + if (item.offset >= part.untilOffset) { + assert(item.offset == part.untilOffset, errOvershotEnd(item.offset, part)) + finished = true + null.asInstanceOf[R] + } else { + requestOffset = item.nextOffset + messageHandler(new MessageAndMetadata( + part.topic, part.partition, item.message, item.offset, keyDecoder, valueDecoder)) + } + } + } + } +} + +private[spark] +object KafkaRDD { + import KafkaCluster.LeaderOffset + + /** + * @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 batch + * @param untilOffsets per-topic/partition Kafka offsets defining the (exclusive) + * ending point of the batch + * @param messageHandler function for translating each message into the desired type + */ + def apply[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag, + R: ClassTag]( + sc: SparkContext, + kafkaParams: Map[String, String], + fromOffsets: Map[TopicAndPartition, Long], + untilOffsets: Map[TopicAndPartition, LeaderOffset], + messageHandler: MessageAndMetadata[K, V] => R + ): KafkaRDD[K, V, U, T, R] = { + val leaders = untilOffsets.map { case (tp, lo) => + tp -> (lo.host, lo.port) + }.toMap + + val offsetRanges = fromOffsets.map { case (tp, fo) => + val uo = untilOffsets(tp) + OffsetRange(tp.topic, tp.partition, fo, uo.offset) + }.toArray + + new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaders, messageHandler) + } +} 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 new file mode 100644 index 0000000000000..36372e08f65f6 --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 org.apache.spark.Partition + +/** @param topic kafka topic name + * @param partition kafka partition id + * @param fromOffset inclusive starting offset + * @param untilOffset exclusive ending offset + * @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] +class KafkaRDDPartition( + val index: Int, + val topic: String, + val partition: Int, + val fromOffset: Long, + val untilOffset: Long, + 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 df725f0c65a64..f8aa6c5c6263c 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 @@ -23,12 +23,18 @@ import java.util.{Map => JMap} import scala.reflect.ClassTag import scala.collection.JavaConversions._ +import kafka.common.TopicAndPartition +import kafka.message.MessageAndMetadata import kafka.serializer.{Decoder, StringDecoder} + +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.dstream.ReceiverInputDStream +import org.apache.spark.streaming.dstream.{InputDStream, ReceiverInputDStream} object KafkaUtils { /** @@ -144,4 +150,174 @@ object KafkaUtils { createStream[K, V, U, T]( 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. + * @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. + * @param offsetRanges Each OffsetRange in the batch corresponds to a + * range of offsets for a given Kafka topic/partition + */ + @Experimental + def createRDD[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag] ( + sc: SparkContext, + kafkaParams: Map[String, String], + 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 + ) + new KafkaRDD[K, V, U, T, (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. + * @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. + * @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 + */ + @Experimental + def createRDD[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag, + R: ClassTag] ( + sc: SparkContext, + kafkaParams: Map[String, String], + offsetRanges: Array[OffsetRange], + leaders: Array[Leader], + messageHandler: MessageAndMetadata[K, V] => 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) + } + + /** + * 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. + * + * 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 + * +. * 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 either ensure that the output operation is + * idempotent, or transactionally store offsets with the output. 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 + */ + @Experimental + def createDirectStream[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: 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]( + 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. + * + * 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. + * + * @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" + * to determine where the stream starts (defaults to "largest") + * @param topics names of the topics to consume + */ + @Experimental + def createDirectStream[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag] ( + ssc: StreamingContext, + kafkaParams: Map[String, String], + topics: Set[String] + ): InputDStream[(K, V)] = { + val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message) + val kc = new KafkaCluster(kafkaParams) + val reset = kafkaParams.get("auto.offset.reset").map(_.toLowerCase) + + (for { + topicPartitions <- kc.getPartitions(topics).right + leaderOffsets <- (if (reset == Some("smallest")) { + kc.getEarliestLeaderOffsets(topicPartitions) + } else { + kc.getLatestLeaderOffsets(topicPartitions) + }).right + } yield { + val fromOffsets = leaderOffsets.map { case (tp, lo) => + (tp, lo.offset) + } + new DirectKafkaInputDStream[K, V, U, T, (K, V)]( + ssc, kafkaParams, fromOffsets, messageHandler) + }).fold( + errs => throw new SparkException(errs.mkString("\n")), + ok => ok + ) + } } 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 new file mode 100644 index 0000000000000..3454d92e72b47 --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Leader.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import kafka.common.TopicAndPartition + +/** Host info for the leader of a Kafka TopicAndPartition */ +final class Leader private( + /** kafka topic name */ + val topic: String, + /** kafka partition id */ + val partition: Int, + /** kafka hostname */ + val host: String, + /** kafka host's port */ + val port: Int) extends Serializable + +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) + + def apply(topicAndPartition: TopicAndPartition, host: String, port: Int): Leader = + new Leader(topicAndPartition.topic, topicAndPartition.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 new file mode 100644 index 0000000000000..334c12e4627b4 --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.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.streaming.kafka + +import kafka.common.TopicAndPartition + +/** Something that has a collection of OffsetRanges */ +trait HasOffsetRanges { + def offsetRanges: Array[OffsetRange] +} + +/** Represents a range of offsets from a single Kafka TopicAndPartition */ +final class OffsetRange private( + /** kafka topic name */ + val topic: String, + /** kafka partition id */ + val partition: Int, + /** inclusive starting offset */ + val fromOffset: Long, + /** exclusive ending offset */ + val untilOffset: Long) extends Serializable { + import OffsetRange.OffsetRangeTuple + + /** this is to avoid ClassNotFoundException during checkpoint restore */ + private[streaming] + def toTuple: OffsetRangeTuple = (topic, partition, fromOffset, untilOffset) +} + +object OffsetRange { + def create(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = + new OffsetRange(topic, partition, fromOffset, untilOffset) + + def create( + topicAndPartition: TopicAndPartition, + fromOffset: Long, + untilOffset: Long): OffsetRange = + new OffsetRange(topicAndPartition.topic, topicAndPartition.partition, fromOffset, untilOffset) + + def apply(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = + new OffsetRange(topic, partition, fromOffset, untilOffset) + + def apply( + topicAndPartition: TopicAndPartition, + fromOffset: Long, + untilOffset: Long): OffsetRange = + new OffsetRange(topicAndPartition.topic, topicAndPartition.partition, fromOffset, untilOffset) + + /** this is to avoid ClassNotFoundException during checkpoint restore */ + private[spark] + type OffsetRangeTuple = (String, Int, Long, Long) + + private[streaming] + def apply(t: OffsetRangeTuple) = + new OffsetRange(t._1, t._2, t._3, t._4) +} 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 new file mode 100644 index 0000000000000..e57c8f6987fdc --- /dev/null +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import scala.util.Random + +import org.scalatest.BeforeAndAfter +import kafka.common.TopicAndPartition + +class KafkaClusterSuite extends KafkaStreamSuiteBase with BeforeAndAfter { + val brokerHost = "localhost" + + val kafkaParams = Map("metadata.broker.list" -> s"$brokerHost:$brokerPort") + + val kc = new KafkaCluster(kafkaParams) + + val topic = "kcsuitetopic" + Random.nextInt(10000) + + val topicAndPartition = TopicAndPartition(topic, 0) + + before { + setupKafka() + createTopic(topic) + produceAndSendMessage(topic, Map("a" -> 1)) + } + + after { + tearDownKafka() + } + + test("metadata apis") { + val leader = kc.findLeaders(Set(topicAndPartition)).right.get + assert(leader(topicAndPartition) === (brokerHost, brokerPort), "didn't get leader") + + val parts = kc.getPartitions(Set(topic)).right.get + assert(parts(topicAndPartition), "didn't get partitions") + } + + test("leader offset apis") { + val earliest = kc.getEarliestLeaderOffsets(Set(topicAndPartition)).right.get + assert(earliest(topicAndPartition).offset === 0, "didn't get earliest") + + val latest = kc.getLatestLeaderOffsets(Set(topicAndPartition)).right.get + assert(latest(topicAndPartition).offset === 1, "didn't get latest") + } + + test("consumer offset apis") { + val group = "kcsuitegroup" + Random.nextInt(10000) + + val offset = Random.nextInt(10000) + + val set = kc.setConsumerOffsets(group, Map(topicAndPartition -> offset)) + assert(set.isRight, "didn't set consumer offsets") + + val get = kc.getConsumerOffsets(group, Set(topicAndPartition)).right.get + assert(get(topicAndPartition) === offset, "didn't get consumer offsets") + } +} 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 new file mode 100644 index 0000000000000..0891ce344f16a --- /dev/null +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaDirectStreamSuite.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.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 new file mode 100644 index 0000000000000..9b9e3f5fce8bd --- /dev/null +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 kafka.serializer.StringDecoder +import kafka.common.TopicAndPartition +import org.scalatest.BeforeAndAfter + +import org.apache.spark._ +import org.apache.spark.SparkContext._ + +class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfter { + var sc: SparkContext = _ + before { + setupKafka() + } + + after { + if (sc != null) { + sc.stop + sc = null + } + tearDownKafka() + } + + test("Kafka RDD") { + val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName) + sc = new SparkContext(sparkConf) + val topic = "topic1" + val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) + createTopic(topic) + produceAndSendMessage(topic, sent) + + val kafkaParams = Map("metadata.broker.list" -> s"localhost:$brokerPort", + "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 + assert(rdd.isDefined) + assert(rdd.get.count === sent.values.sum) + + kc.setConsumerOffsets( + kafkaParams("group.id"), + rdd.get.offsetRanges.map(o => TopicAndPartition(o.topic, o.partition) -> o.untilOffset).toMap) + + val rdd2 = getRdd(kc, Set(topic)) + val sent2 = Map("d" -> 1) + produceAndSendMessage(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)) + // 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) + + } + + // 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( + 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}") + } + } +} 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 0817c56d8f39f..f207dc6d4fa04 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 @@ -26,7 +26,7 @@ import scala.concurrent.duration._ import scala.language.postfixOps import scala.util.Random -import kafka.admin.CreateTopicCommand +import kafka.admin.AdminUtils import kafka.common.{KafkaException, TopicAndPartition} import kafka.producer.{KeyedMessage, Producer, ProducerConfig} import kafka.serializer.{StringDecoder, StringEncoder} @@ -56,7 +56,7 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Loggin private val zkSessionTimeout = 6000 private var zookeeper: EmbeddedZookeeper = _ private var zkPort: Int = 0 - private var brokerPort = 9092 + protected var brokerPort = 9092 private var brokerConf: KafkaConfig = _ private var server: KafkaServer = _ private var producer: Producer[String, String] = _ @@ -130,7 +130,7 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Loggin } def createTopic(topic: String) { - CreateTopicCommand.createTopic(zkClient, topic, 1, 1, "0") + AdminUtils.createTopic(zkClient, topic, 1, 1) logInfo("==================== 5 ====================") // wait until metadata is propagated waitUntilMetadataIsPropagated(topic, 0) @@ -166,7 +166,7 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Loggin private def waitUntilMetadataIsPropagated(topic: String, partition: Int) { eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { assert( - server.apis.leaderCache.keySet.contains(TopicAndPartition(topic, partition)), + server.apis.metadataCache.containsTopicAndPartition(topic, partition), s"Partition [$topic, $partition] metadata not propagated after timeout" ) } From f0500f9fa378d81e4b4038a66a40eee15806b677 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 4 Feb 2015 14:20:44 -0800 Subject: [PATCH 066/578] [SPARK-4707][STREAMING] Reliable Kafka Receiver can lose data if the blo... ...ck generator fails to store data. The Reliable Kafka Receiver commits offsets only when events are actually stored, which ensures that on restart we will actually start where we left off. But if the failure happens in the store() call, and the block generator reports an error the receiver does not do anything and will continue reading from the current offset and not the last commit. This means that messages between the last commit and the current offset will be lost. This PR retries the store call four times and then stops the receiver with an error message and the last exception that was received from the store. Author: Hari Shreedharan Closes #3655 from harishreedharan/kafka-failure-fix and squashes the following commits: 5e2e7ad [Hari Shreedharan] [SPARK-4704][STREAMING] Reliable Kafka Receiver can lose data if the block generator fails to store data. --- .../kafka/ReliableKafkaReceiver.scala | 27 ++++++++++++++++--- 1 file changed, 23 insertions(+), 4 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala index be734b80272d1..c4a44c1822c39 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -201,12 +201,31 @@ class ReliableKafkaReceiver[ topicPartitionOffsetMap.clear() } - /** Store the ready-to-be-stored block and commit the related offsets to zookeeper. */ + /** + * Store the ready-to-be-stored block and commit the related offsets to zookeeper. This method + * will try a fixed number of times to push the block. If the push fails, the receiver is stopped. + */ private def storeBlockAndCommitOffset( blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { - store(arrayBuffer.asInstanceOf[mutable.ArrayBuffer[(K, V)]]) - Option(blockOffsetMap.get(blockId)).foreach(commitOffset) - blockOffsetMap.remove(blockId) + var count = 0 + var pushed = false + var exception: Exception = null + while (!pushed && count <= 3) { + try { + store(arrayBuffer.asInstanceOf[mutable.ArrayBuffer[(K, V)]]) + pushed = true + } catch { + case ex: Exception => + count += 1 + exception = ex + } + } + if (pushed) { + Option(blockOffsetMap.get(blockId)).foreach(commitOffset) + blockOffsetMap.remove(blockId) + } else { + stop("Error while storing block into Spark", exception) + } } /** From 0a89b156850fc5ba93160987927f249a7e633d51 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 4 Feb 2015 14:22:07 -0800 Subject: [PATCH 067/578] [SPARK-4939] move to next locality when no pending tasks Currently, if there are different locality in a task set, the tasks with NODE_LOCAL only get scheduled after all the PROCESS_LOCAL tasks are scheduled and timeout with spark.locality.wait.process (3 seconds by default). In local mode, the LocalScheduler will never call resourceOffer() again once it failed to get a task with same locality, then all the NODE_LOCAL tasks will be never scheduled. This bug could be reproduced by run example python/streaming/stateful_network_wordcount.py, it will hang after finished a batch with some data. This patch will check whether there is task for current locality level, if not, it will change to next locality level without waiting for `spark.locality.wait.process` seconds. It works for all locality levels. Because the list of pending tasks are updated lazily, the check can be false-positive, it means it will not move to next locality level even there is no valid pending tasks, it will wait for timeout. Author: Davies Liu Closes #3779 from davies/local_streaming and squashes the following commits: 2d25fb3 [Davies Liu] Update TaskSetManager.scala 1550668 [Davies Liu] add comment 1c37aac [Davies Liu] address comments 6b13824 [Davies Liu] address comments 906f456 [Davies Liu] Merge branch 'master' of github.com:apache/spark into local_streaming 414e79e [Davies Liu] fix bug, add logging ff8eabb [Davies Liu] Merge branch 'master' into local_streaming 28d1b3c [Davies Liu] check tasks 9d0ceab [Davies Liu] Merge branch 'master' of github.com:apache/spark into local_streaming 37a2804 [Davies Liu] fix tests 49bda82 [Davies Liu] address comment d8fb95a [Davies Liu] move to next locality level if no more tasks 2d6ae73 [Davies Liu] add comments 32d363f [Davies Liu] add regression test 7d8c5a5 [Davies Liu] jump to next locality if no pending tasks for executors --- .../spark/scheduler/TaskSetManager.scala | 65 +++++++++++++++++-- .../spark/scheduler/TaskSetManagerSuite.scala | 44 ++++++++++++- 2 files changed, 101 insertions(+), 8 deletions(-) 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 97c22fe724abd..55024ecd55e61 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -506,13 +506,64 @@ private[spark] class TaskSetManager( * Get the level we can launch tasks according to delay scheduling, based on current wait time. */ private def getAllowedLocalityLevel(curTime: Long): TaskLocality.TaskLocality = { - while (curTime - lastLaunchTime >= localityWaits(currentLocalityIndex) && - currentLocalityIndex < myLocalityLevels.length - 1) - { - // Jump to the next locality level, and remove our waiting time for the current one since - // we don't want to count it again on the next one - lastLaunchTime += localityWaits(currentLocalityIndex) - currentLocalityIndex += 1 + // Remove the scheduled or finished tasks lazily + def tasksNeedToBeScheduledFrom(pendingTaskIds: ArrayBuffer[Int]): Boolean = { + var indexOffset = pendingTaskIds.size + while (indexOffset > 0) { + indexOffset -= 1 + val index = pendingTaskIds(indexOffset) + if (copiesRunning(index) == 0 && !successful(index)) { + return true + } else { + pendingTaskIds.remove(indexOffset) + } + } + false + } + // Walk through the list of tasks that can be scheduled at each location and returns true + // if there are any tasks that still need to be scheduled. Lazily cleans up tasks that have + // already been scheduled. + def moreTasksToRunIn(pendingTasks: HashMap[String, ArrayBuffer[Int]]): Boolean = { + val emptyKeys = new ArrayBuffer[String] + val hasTasks = pendingTasks.exists { + case (id: String, tasks: ArrayBuffer[Int]) => + if (tasksNeedToBeScheduledFrom(tasks)) { + true + } else { + emptyKeys += id + false + } + } + // The key could be executorId, host or rackId + emptyKeys.foreach(id => pendingTasks.remove(id)) + hasTasks + } + + while (currentLocalityIndex < myLocalityLevels.length - 1) { + val moreTasks = myLocalityLevels(currentLocalityIndex) match { + case TaskLocality.PROCESS_LOCAL => moreTasksToRunIn(pendingTasksForExecutor) + case TaskLocality.NODE_LOCAL => moreTasksToRunIn(pendingTasksForHost) + case TaskLocality.NO_PREF => pendingTasksWithNoPrefs.nonEmpty + case TaskLocality.RACK_LOCAL => moreTasksToRunIn(pendingTasksForRack) + } + if (!moreTasks) { + // This is a performance optimization: if there are no more tasks that can + // be scheduled at a particular locality level, there is no point in waiting + // for the locality wait timeout (SPARK-4939). + lastLaunchTime = curTime + logDebug(s"No tasks for locality level ${myLocalityLevels(currentLocalityIndex)}, " + + s"so moving to locality level ${myLocalityLevels(currentLocalityIndex + 1)}") + currentLocalityIndex += 1 + } else if (curTime - lastLaunchTime >= localityWaits(currentLocalityIndex)) { + // Jump to the next locality level, and reset lastLaunchTime so that the next locality + // wait timer doesn't immediately expire + lastLaunchTime += localityWaits(currentLocalityIndex) + currentLocalityIndex += 1 + logDebug(s"Moving to ${myLocalityLevels(currentLocalityIndex)} after waiting for " + + s"${localityWaits(currentLocalityIndex)}ms") + } else { + return myLocalityLevels(currentLocalityIndex) + } } myLocalityLevels(currentLocalityIndex) } 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 84b9b788237bf..59580561cb45a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -314,7 +314,8 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("delay scheduling with failed hosts") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) + val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2"), + ("exec3", "host3")) val taskSet = FakeTask.createTaskSet(3, Seq(TaskLocation("host1")), Seq(TaskLocation("host2")), @@ -649,6 +650,47 @@ 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") { + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2")) + val taskSet = FakeTask.createTaskSet(4, + Seq(TaskLocation("host1")), + Seq(TaskLocation("host2")), + Seq(ExecutorCacheTaskLocation("host1", "execA")), + Seq(ExecutorCacheTaskLocation("host2", "execB"))) + val clock = new FakeClock + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + + // process-local tasks are scheduled first + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 2) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL).get.index === 3) + // node-local tasks are scheduled without delay + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 0) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL).get.index === 1) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) + assert(manager.resourceOffer("execB", "host2", NODE_LOCAL) == None) + } + + test("SPARK-4939: no-pref tasks should be scheduled after process-local tasks finished") { + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2")) + val taskSet = FakeTask.createTaskSet(3, + Seq(), + Seq(ExecutorCacheTaskLocation("host1", "execA")), + Seq(ExecutorCacheTaskLocation("host2", "execB"))) + val clock = new FakeClock + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + + // process-local tasks are scheduled first + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 1) + assert(manager.resourceOffer("execB", "host2", PROCESS_LOCAL).get.index === 2) + // no-pref tasks are scheduled without delay + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL) == None) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) + assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index === 0) + assert(manager.resourceOffer("execA", "host1", ANY) == None) + } + test("Ensure TaskSetManager is usable after addition of levels") { // Regression test for SPARK-2931 sc = new SparkContext("local", "test") From a9f0db1fb41a6e5510d2c14f79ff086da2ecc7b0 Mon Sep 17 00:00:00 2001 From: wangfei Date: Wed, 4 Feb 2015 14:33:07 -0800 Subject: [PATCH 068/578] [SPARK-5591][SQL] Fix NoSuchObjectException for CTAS Now CTAS runs successfully but will throw a NoSuchObjectException. ``` 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; ``` Get this exception: ERROR Hive: NoSuchObjectException(message:default.sc table not found) at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.get_table(HiveMetaStore.java:1560) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:601) at org.apache.hadoop.hive.metastore.RetryingHMSHandler.invoke(RetryingHMSHandler.java:105) at $Proxy8.get_table(Unknown Source) at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.getTable(HiveMetaStoreClient.java:997) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:601) at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:89) at $Proxy9.getTable(Unknown Source) at org.apache.hadoop.hive.ql.metadata.Hive.getTable(Hive.java:976) at org.apache.hadoop.hive.ql.metadata.Hive.getTable(Hive.java:950) at org.apache.spark.sql.hive.HiveMetastoreCatalog.tableExists(HiveMetastoreCatalog.scala:152) at org.apache.spark.sql.hive.HiveContext$$anon$2.org$apache$spark$sql$catalyst$analysis$OverrideCatalog$$super$tableExists(HiveContext.scala:309) at org.apache.spark.sql.catalyst.analysis.OverrideCatalog$class.tableExists(Catalog.scala:121) at org.apache.spark.sql.hive.HiveContext$$anon$2.tableExists(HiveContext.scala:309) at org.apache.spark.sql.hive.execution.CreateTableAsSelect.run(CreateTableAsSelect.scala:63) at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:53) Author: wangfei Closes #4365 from scwf/ctas-exception and squashes the following commits: c7c67bc [wangfei] no used imports f54eb2a [wangfei] fix exception for CTAS --- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 8 ++------ 1 file changed, 2 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 48bea6c1bd685..bcb4726e90a77 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 @@ -24,7 +24,7 @@ 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, AlreadyExistsException, FieldSchema} +import org.apache.hadoop.hive.metastore.api.{Table => TTable, Partition => TPartition, FieldSchema} import org.apache.hadoop.hive.ql.metadata._ import org.apache.hadoop.hive.ql.plan.CreateTableDesc import org.apache.hadoop.hive.serde.serdeConstants @@ -148,11 +148,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val databaseName = tableIdent.lift(tableIdent.size - 2).getOrElse( hive.sessionState.getCurrentDatabase) val tblName = tableIdent.last - try { - client.getTable(databaseName, tblName) != null - } catch { - case ie: InvalidTableException => false - } + client.getTable(databaseName, tblName, false) != null } def lookupRelation( From b90dd39793ab0a13a4559fcfff3bb5305c92ab3b Mon Sep 17 00:00:00 2001 From: wangfei Date: Wed, 4 Feb 2015 14:35:12 -0800 Subject: [PATCH 069/578] [SPARK-5587][SQL] Support change database owner Support change database owner, here i do not add the golden files since the golden answer is related to the tmp dir path (see https://github.com/scwf/spark/commit/6331e4ac0f982caf70531defcb957be76fe093c7) Author: wangfei Closes #4357 from scwf/db_owner and squashes the following commits: f761533 [wangfei] remove the alter_db_owner which have added to whitelist 79413c6 [wangfei] Revert "added golden files" 6331e4a [wangfei] added golden files 6f7cacd [wangfei] support change database owner --- 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 74ca0d4ed5bb1..9b5c081557d8b 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 @@ -95,6 +95,7 @@ private[hive] object HiveQl { "TOK_DROPFUNCTION", "TOK_ALTERDATABASE_PROPERTIES", + "TOK_ALTERDATABASE_OWNER", "TOK_ALTERINDEX_PROPERTIES", "TOK_ALTERINDEX_REBUILD", "TOK_ALTERTABLE_ADDCOLS", From 424cb699ee9b091eb23b86dc018a86e377ad309f Mon Sep 17 00:00:00 2001 From: kul Date: Wed, 4 Feb 2015 15:08:37 -0800 Subject: [PATCH 070/578] [SPARK-5426][SQL] Add SparkSQL Java API helper methods. Right now the PR adds few helper methods for java apis. But the issue was opened mainly to get rid of transformations in java api like `.rdd` and `.toJavaRDD` while working with `SQLContext` or `HiveContext`. Author: kul Closes #4243 from kul/master and squashes the following commits: 2390fba [kul] [SPARK-5426][SQL] Add SparkSQL Java API helper methods. --- .../org/apache/spark/sql/SQLContext.scala | 17 +++++++ .../sql/api/java/JavaApplySchemaSuite.java | 48 +++++++++++++++++-- 2 files changed, 62 insertions(+), 3 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 2697e780c05c6..1661282fc31c4 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 @@ -221,6 +221,11 @@ class SQLContext(@transient val sparkContext: SparkContext) DataFrame(this, logicalPlan) } + @DeveloperApi + def applySchema(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = { + applySchema(rowRDD.rdd, schema); + } + /** * Applies a schema to an RDD of Java Beans. * @@ -305,6 +310,8 @@ class SQLContext(@transient val sparkContext: SparkContext) */ def jsonRDD(json: RDD[String]): DataFrame = jsonRDD(json, 1.0) + def jsonRDD(json: JavaRDD[String]): DataFrame = jsonRDD(json.rdd, 1.0) + /** * :: Experimental :: * Loads an RDD[String] storing JSON objects (one object per record) and applies the given schema, @@ -323,6 +330,11 @@ class SQLContext(@transient val sparkContext: SparkContext) applySchema(rowRDD, appliedSchema) } + @Experimental + def jsonRDD(json: JavaRDD[String], schema: StructType): DataFrame = { + jsonRDD(json.rdd, schema) + } + /** * :: Experimental :: */ @@ -336,6 +348,11 @@ class SQLContext(@transient val sparkContext: SparkContext) applySchema(rowRDD, appliedSchema) } + @Experimental + def jsonRDD(json: JavaRDD[String], samplingRatio: Double): DataFrame = { + jsonRDD(json.rdd, samplingRatio); + } + @Experimental def load(path: String): DataFrame = { val dataSourceName = conf.defaultDataSourceName 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 badd00d34b9b1..8510bac499092 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 @@ -98,7 +98,7 @@ 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.rdd(), schema); + DataFrame df = javaSqlCtx.applySchema(rowRDD, schema); df.registerTempTable("people"); Row[] actual = javaSqlCtx.sql("SELECT * FROM people").collect(); @@ -109,6 +109,48 @@ public Row call(Person person) throws Exception { Assert.assertEquals(expected, Arrays.asList(actual)); } + + + @Test + public void dataFrameRDDOperations() { + List personList = new ArrayList(2); + Person person1 = new Person(); + person1.setName("Michael"); + person1.setAge(29); + personList.add(person1); + Person person2 = new Person(); + person2.setName("Yin"); + person2.setAge(28); + personList.add(person2); + + JavaRDD rowRDD = javaCtx.parallelize(personList).map( + new Function() { + public Row call(Person person) throws Exception { + return RowFactory.create(person.getName(), person.getAge()); + } + }); + + List fields = new ArrayList(2); + fields.add(DataTypes.createStructField("name", DataTypes.StringType, false)); + fields.add(DataTypes.createStructField("age", DataTypes.IntegerType, false)); + StructType schema = DataTypes.createStructType(fields); + + DataFrame df = javaSqlCtx.applySchema(rowRDD, schema); + df.registerTempTable("people"); + List actual = javaSqlCtx.sql("SELECT * FROM people").toJavaRDD().map(new Function() { + + public String call(Row row) { + return row.getString(0) + "_" + row.get(1).toString(); + } + }).collect(); + + List expected = new ArrayList(2); + expected.add("Michael_29"); + expected.add("Yin_28"); + + Assert.assertEquals(expected, actual); + } + @Test public void applySchemaToJSON() { JavaRDD jsonRDD = javaCtx.parallelize(Arrays.asList( @@ -147,14 +189,14 @@ public void applySchemaToJSON() { null, "this is another simple string.")); - DataFrame df1 = javaSqlCtx.jsonRDD(jsonRDD.rdd()); + DataFrame df1 = javaSqlCtx.jsonRDD(jsonRDD); StructType actualSchema1 = df1.schema(); Assert.assertEquals(expectedSchema, actualSchema1); df1.registerTempTable("jsonTable1"); List actual1 = javaSqlCtx.sql("select * from jsonTable1").collectAsList(); Assert.assertEquals(expectedResult, actual1); - DataFrame df2 = javaSqlCtx.jsonRDD(jsonRDD.rdd(), expectedSchema); + DataFrame df2 = javaSqlCtx.jsonRDD(jsonRDD, expectedSchema); StructType actualSchema2 = df2.schema(); Assert.assertEquals(expectedSchema, actualSchema2); df2.registerTempTable("jsonTable2"); From 417d1118cd720b616bfd30e9c1a0bfb1838e8cbd Mon Sep 17 00:00:00 2001 From: wangfei Date: Wed, 4 Feb 2015 15:12:07 -0800 Subject: [PATCH 071/578] [SPARK-5367][SQL] Support star expression in udfs A follow up for #4163: support `select array(key, *) from src` Since array(key, *) will not go into this case ``` case Alias(f UnresolvedFunction(_, args), name) if containsStar(args) => val expandedArgs = args.flatMap { case s: Star => s.expand(child.output, resolver) case o => o :: Nil } ``` here added a case to cover the corner case of array. /cc liancheng Author: wangfei Author: scwf Closes #4353 from scwf/udf-star1 and squashes the following commits: 4350d17 [wangfei] minor fix a7cd191 [wangfei] minor fix 0942fb1 [wangfei] follow up: support select array(key, *) from src 6ae00db [wangfei] also fix problem with array da1da09 [scwf] minor fix f87b5f9 [scwf] added test case 587bf7e [wangfei] compile fix eb93c16 [wangfei] fix star resolve issue in udf --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 6 ++++++ .../apache/spark/sql/hive/execution/HiveQuerySuite.scala | 2 ++ 2 files changed, 8 insertions(+) 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 ae7f7b9feb5fd..ae1aee02c64a5 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 @@ -257,6 +257,12 @@ class Analyzer(catalog: Catalog, case o => o :: Nil } Alias(child = f.copy(children = expandedArgs), name)() :: Nil + case Alias(c @ CreateArray(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/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 82efadb28e890..a321452cef74f 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 @@ -620,6 +620,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { test("SPARK-5367: resolve star expression in udf") { assert(sql("select concat(*) from src limit 5").collect().size == 5) assert(sql("select array(*) from src limit 5").collect().size == 5) + assert(sql("select concat(key, *) from src limit 5").collect().size == 5) + assert(sql("select array(key, *) from src limit 5").collect().size == 5) } test("Query Hive native command execution result") { From b73d5fff7208f9d09adde92c241c70095983bc09 Mon Sep 17 00:00:00 2001 From: OopsOutOfMemory Date: Wed, 4 Feb 2015 15:13:54 -0800 Subject: [PATCH 072/578] [SQL][Hiveconsole] Bring hive console code up to date and update README.md Add `import org.apache.spark.sql.Dsl._` to make DSL query works. Since queryExecution is not avaliable in DataFrame, so remove it. Author: OopsOutOfMemory Author: Sheng, Li Closes #4330 from OopsOutOfMemory/hiveconsole and squashes the following commits: 46eb790 [Sheng, Li] Update SparkBuild.scala d23ee9f [OopsOutOfMemory] minor d4dd593 [OopsOutOfMemory] refine hive console --- project/SparkBuild.scala | 1 + sql/README.md | 45 +++++++++------------------------------- 2 files changed, 11 insertions(+), 35 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index fbc8983b953b7..93698efe84252 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -245,6 +245,7 @@ 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.test.TestSQLContext._ |import org.apache.spark.sql.types._ diff --git a/sql/README.md b/sql/README.md index 61a20916a92aa..254ab8eb6376f 100644 --- a/sql/README.md +++ b/sql/README.md @@ -22,59 +22,34 @@ export HADOOP_HOME="/hadoop-1.0.4" Using the console ================= -An interactive scala console can be invoked by running `build/sbt hive/console`. From here you can execute queries and inspect the various stages of query optimization. +An interactive scala console can be invoked by running `build/sbt hive/console`. +From here you can execute queries with HiveQl and manipulate DataFrame by using DSL. ```scala catalyst$ build/sbt hive/console [info] Starting scala interpreter... -import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.dsl._ -import org.apache.spark.sql.catalyst.errors._ -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.execution +import org.apache.spark.sql.Dsl._ import org.apache.spark.sql.hive._ -import org.apache.spark.sql.hive.TestHive._ +import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.types._ +import org.apache.spark.sql.parquet.ParquetTestData Welcome to Scala version 2.10.4 (Java HotSpot(TM) 64-Bit Server VM, Java 1.7.0_45). Type in expressions to have them evaluated. Type :help for more information. scala> val query = sql("SELECT * FROM (SELECT * FROM src) a") -query: org.apache.spark.sql.DataFrame = -== Query Plan == -== Physical Plan == -HiveTableScan [key#10,value#11], (MetastoreRelation default, src, None), None +query: org.apache.spark.sql.DataFrame = org.apache.spark.sql.DataFrame@74448eed ``` -Query results are RDDs and can be operated as such. +Query results are `DataFrames` and can be operated as such. ``` scala> query.collect() res2: Array[org.apache.spark.sql.Row] = Array([238,val_238], [86,val_86], [311,val_311], [27,val_27]... ``` -You can also build further queries on top of these RDDs using the query DSL. +You can also build further queries on top of these `DataFrames` using the query DSL. ``` -scala> query.where('key === 100).collect() -res3: Array[org.apache.spark.sql.Row] = Array([100,val_100], [100,val_100]) -``` - -From the console you can even write rules that transform query plans. For example, the above query has redundant project operators that aren't doing anything. This redundancy can be eliminated using the `transform` function that is available on all [`TreeNode`](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala) objects. -```scala -scala> query.queryExecution.analyzed -res4: org.apache.spark.sql.catalyst.plans.logical.LogicalPlan = -Project [key#10,value#11] - Project [key#10,value#11] - MetastoreRelation default, src, None - - -scala> query.queryExecution.analyzed transform { - | case Project(projectList, child) if projectList == child.output => child - | } -res5: res17: org.apache.spark.sql.catalyst.plans.logical.LogicalPlan = -Project [key#10,value#11] - MetastoreRelation default, src, None +scala> query.where('key > 30).select(avg('key)).collect() +res3: Array[org.apache.spark.sql.Row] = Array([274.79025423728814]) ``` From 0d81645f776aeca6a48322da392d4fac18431556 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 4 Feb 2015 15:14:49 -0800 Subject: [PATCH 073/578] [SQL] Correct the default size of TimestampType and expose NumericType Author: Yin Huai Closes #4314 from yhuai/minor and squashes the following commits: d3870a7 [Yin Huai] Update test. 6e4b0c0 [Yin Huai] Two minor changes. --- .../main/scala/org/apache/spark/sql/types/dataTypes.scala | 6 +++--- .../scala/org/apache/spark/sql/types/DataTypeSuite.scala | 2 +- 2 files changed, 4 insertions(+), 4 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 4825d1ff81402..a6d6ddd905393 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 @@ -371,9 +371,9 @@ case object TimestampType extends NativeType { } /** - * The default size of a value of the TimestampType is 8 bytes. + * The default size of a value of the TimestampType is 12 bytes. */ - override def defaultSize: Int = 8 + override def defaultSize: Int = 12 } @@ -400,7 +400,7 @@ case object DateType extends NativeType { } -protected[sql] abstract class NumericType extends NativeType with PrimitiveType { +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 // type parameter and and add a numeric annotation (i.e., [JvmType : Numeric]). This gets 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 7bcd6687d11a1..c97e0bec3e3a2 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 @@ -107,7 +107,7 @@ class DataTypeSuite extends FunSuite { checkDefaultSize(DecimalType(10, 5), 4096) checkDefaultSize(DecimalType.Unlimited, 4096) checkDefaultSize(DateType, 4) - checkDefaultSize(TimestampType, 8) + checkDefaultSize(TimestampType,12) checkDefaultSize(StringType, 4096) checkDefaultSize(BinaryType, 4096) checkDefaultSize(ArrayType(DoubleType, true), 800) From 548c9c2b2a1612f5e4342de322b72b9e78140618 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 4 Feb 2015 15:22:40 -0800 Subject: [PATCH 074/578] [SQL] Use HiveContext's sessionState in HiveMetastoreCatalog.hiveDefaultTableFilePath `client.getDatabaseCurrent` uses SessionState's local variable which can be an issue. Author: Yin Huai Closes #4355 from yhuai/defaultTablePath and squashes the following commits: 84a29e5 [Yin Huai] Use HiveContext's sessionState instead of using SessionState's thread local variable. --- .../scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index bcb4726e90a77..243310686d08a 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 @@ -140,7 +140,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } def hiveDefaultTableFilePath(tableName: String): String = { - hiveWarehouse.getTablePath(client.getDatabaseCurrent, tableName).toString + val currentDatabase = client.getDatabase(hive.sessionState.getCurrentDatabase()) + hiveWarehouse.getTablePath(currentDatabase, tableName).toString } def tableExists(tableIdentifier: Seq[String]): Boolean = { From e0490e271d078aa55d7c7583e2ba80337ed1b0c4 Mon Sep 17 00:00:00 2001 From: guowei2 Date: Wed, 4 Feb 2015 15:26:10 -0800 Subject: [PATCH 075/578] [SPARK-5118][SQL] Fix: create table test stored as parquet as select .. Author: guowei2 Closes #3921 from guowei2/SPARK-5118 and squashes the following commits: b1ba3be [guowei2] add table file check in test case 9da56f8 [guowei2] test case only run in Shim13 112a0b6 [guowei2] add test case 187c7d8 [guowei2] Fix: create table test stored as parquet as select .. --- .../org/apache/spark/sql/hive/HiveQl.scala | 1 + .../sql/hive/execution/SQLQuerySuite.scala | 29 ++++++++++++++++++- 2 files changed, 29 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 9b5c081557d8b..286d13944c892 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 @@ -554,6 +554,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C "TOK_TBLTEXTFILE", // Stored as TextFile "TOK_TBLRCFILE", // Stored as RCFile "TOK_TBLORCFILE", // Stored as ORC File + "TOK_TBLPARQUETFILE", // Stored as PARQUET "TOK_TABLEFILEFORMAT", // User-provided InputFormat and OutputFormat "TOK_STORAGEHANDLER", // Storage handler "TOK_TABLELOCATION", 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 4efe0c5e0cd44..268e5f6f01dff 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,6 +22,8 @@ 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._ +import org.apache.spark.util.Utils +import org.apache.spark.sql.hive.HiveShim case class Nested1(f1: Nested2) case class Nested2(f2: Nested3) @@ -102,6 +104,31 @@ class SQLQuerySuite extends QueryTest { "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe", "serde_p1=p1", "serde_p2=p2", "tbl_p1=p11", "tbl_p2=p22","MANAGED_TABLE" ) + + if (HiveShim.version =="0.13.1") { + sql( + """CREATE TABLE ctas5 + | STORED AS parquet AS + | SELECT key, value + | FROM src + | ORDER BY key, value""".stripMargin).collect + + checkExistence(sql("DESC EXTENDED ctas5"), true, + "name:key", "type:string", "name:value", "ctas5", + "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat", + "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat", + "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe", + "MANAGED_TABLE" + ) + + val default = getConf("spark.sql.hive.convertMetastoreParquet", "true") + // use the Hive SerDe for parquet tables + sql("set spark.sql.hive.convertMetastoreParquet = false") + checkAnswer( + sql("SELECT key, value FROM ctas5 ORDER BY key, value"), + sql("SELECT key, value FROM src ORDER BY key, value").collect().toSeq) + sql(s"set spark.sql.hive.convertMetastoreParquet = $default") + } } test("command substitution") { @@ -159,7 +186,7 @@ class SQLQuerySuite extends QueryTest { test("test CTAS") { checkAnswer(sql("CREATE TABLE test_ctas_123 AS SELECT key, value FROM src"), Seq.empty[Row]) checkAnswer( - sql("SELECT key, value FROM test_ctas_123 ORDER BY key"), + sql("SELECT key, value FROM test_ctas_123 ORDER BY key"), sql("SELECT key, value FROM src ORDER BY key").collect().toSeq) } From dc101b0e4e23dffddbc2f70d14a19fae5d87a328 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 4 Feb 2015 15:55:09 -0800 Subject: [PATCH 076/578] [SPARK-5577] Python udf for DataFrame Author: Davies Liu Closes #4351 from davies/python_udf and squashes the following commits: d250692 [Davies Liu] fix conflict 34234d4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python_udf 440f769 [Davies Liu] address comments f0a3121 [Davies Liu] track life cycle of broadcast f99b2e1 [Davies Liu] address comments 462b334 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python_udf 7bccc3b [Davies Liu] python udf 58dee20 [Davies Liu] clean up --- python/pyspark/rdd.py | 38 ++-- python/pyspark/sql.py | 195 ++++++++---------- .../scala/org/apache/spark/sql/Column.scala | 19 +- .../spark/sql/UserDefinedFunction.scala | 27 +++ 4 files changed, 157 insertions(+), 122 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 2f8a0edfe9644..6e029bf7f13fc 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2162,6 +2162,25 @@ def toLocalIterator(self): yield row +def _prepare_for_python_RDD(sc, command, obj=None): + # the serialized command will be compressed by broadcast + ser = CloudPickleSerializer() + pickled_command = ser.dumps(command) + if len(pickled_command) > (1 << 20): # 1M + 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) + sc._pickled_broadcast_vars.clear() + env = MapConverter().convert(sc.environment, sc._gateway._gateway_client) + includes = ListConverter().convert(sc._python_includes, sc._gateway._gateway_client) + return pickled_command, broadcast_vars, env, includes + + class PipelinedRDD(RDD): """ @@ -2228,25 +2247,12 @@ def _jrdd(self): command = (self.func, profiler, self._prev_jrdd_deserializer, self._jrdd_deserializer) - # the serialized command will be compressed by broadcast - ser = CloudPickleSerializer() - pickled_command = ser.dumps(command) - if len(pickled_command) > (1 << 20): # 1M - self._broadcast = self.ctx.broadcast(pickled_command) - pickled_command = ser.dumps(self._broadcast) - broadcast_vars = ListConverter().convert( - [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], - self.ctx._gateway._gateway_client) - self.ctx._pickled_broadcast_vars.clear() - env = MapConverter().convert(self.ctx.environment, - self.ctx._gateway._gateway_client) - includes = ListConverter().convert(self.ctx._python_includes, - self.ctx._gateway._gateway_client) + pickled_cmd, bvars, env, includes = _prepare_for_python_RDD(self.ctx, command, self) python_rdd = self.ctx._jvm.PythonRDD(self._prev_jrdd.rdd(), - bytearray(pickled_command), + bytearray(pickled_cmd), env, includes, self.preservesPartitioning, self.ctx.pythonExec, - broadcast_vars, self.ctx._javaAccumulator) + bvars, self.ctx._javaAccumulator) self._jrdd_val = python_rdd.asJavaRDD() if profiler: diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index a266cde51d317..5b56b36bdcdb7 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -51,7 +51,7 @@ from py4j.java_collections import ListConverter, MapConverter from pyspark.context import SparkContext -from pyspark.rdd import RDD +from pyspark.rdd import RDD, _prepare_for_python_RDD from pyspark.serializers import BatchedSerializer, AutoBatchedSerializer, PickleSerializer, \ CloudPickleSerializer, UTF8Deserializer from pyspark.storagelevel import StorageLevel @@ -1274,28 +1274,15 @@ def registerFunction(self, name, f, returnType=StringType()): [Row(c0=4)] """ func = lambda _, it: imap(lambda x: f(*x), it) - command = (func, None, - AutoBatchedSerializer(PickleSerializer()), - AutoBatchedSerializer(PickleSerializer())) - ser = CloudPickleSerializer() - pickled_command = ser.dumps(command) - if len(pickled_command) > (1 << 20): # 1M - broadcast = self._sc.broadcast(pickled_command) - pickled_command = ser.dumps(broadcast) - broadcast_vars = ListConverter().convert( - [x._jbroadcast for x in self._sc._pickled_broadcast_vars], - self._sc._gateway._gateway_client) - self._sc._pickled_broadcast_vars.clear() - env = MapConverter().convert(self._sc.environment, - self._sc._gateway._gateway_client) - includes = ListConverter().convert(self._sc._python_includes, - self._sc._gateway._gateway_client) + 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_command), + bytearray(pickled_cmd), env, includes, self._sc.pythonExec, - broadcast_vars, + bvars, self._sc._javaAccumulator, returnType.json()) @@ -2077,9 +2064,9 @@ def dtypes(self): """Return all column names and their data types as a list. >>> df.dtypes - [(u'age', 'IntegerType'), (u'name', 'StringType')] + [('age', 'integer'), ('name', 'string')] """ - return [(f.name, str(f.dataType)) for f in self.schema().fields] + return [(str(f.name), f.dataType.jsonValue()) for f in self.schema().fields] @property def columns(self): @@ -2194,7 +2181,7 @@ def select(self, *cols): [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).As('age')).collect() + >>> 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: @@ -2295,25 +2282,13 @@ def subtract(self, other): """ return DataFrame(getattr(self._jdf, "except")(other._jdf), self.sql_ctx) - def sample(self, withReplacement, fraction, seed=None): - """ Return a new DataFrame by sampling a fraction of rows. - - >>> df.sample(False, 0.5, 10).collect() - [Row(age=2, name=u'Alice')] - """ - if seed is None: - jdf = self._jdf.sample(withReplacement, fraction) - else: - jdf = self._jdf.sample(withReplacement, fraction, seed) - return DataFrame(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.As(colName)) + return self.select('*', col.alias(colName)) # Having SchemaRDD for backward compatibility (for docs) @@ -2408,28 +2383,6 @@ def sum(self): group.""" -SCALA_METHOD_MAPPINGS = { - '=': '$eq', - '>': '$greater', - '<': '$less', - '+': '$plus', - '-': '$minus', - '*': '$times', - '/': '$div', - '!': '$bang', - '@': '$at', - '#': '$hash', - '%': '$percent', - '^': '$up', - '&': '$amp', - '~': '$tilde', - '?': '$qmark', - '|': '$bar', - '\\': '$bslash', - ':': '$colon', -} - - def _create_column_from_literal(literal): sc = SparkContext._active_spark_context return sc._jvm.Dsl.lit(literal) @@ -2448,23 +2401,18 @@ def _to_java_column(col): return jcol -def _scalaMethod(name): - """ Translate operators into methodName in Scala - - >>> _scalaMethod('+') - '$plus' - >>> _scalaMethod('>=') - '$greater$eq' - >>> _scalaMethod('cast') - 'cast' - """ - return ''.join(SCALA_METHOD_MAPPINGS.get(c, c) for c in name) - - def _unary_op(name, doc="unary operator"): """ Create a method for given unary operator """ def _(self): - jc = getattr(self._jc, _scalaMethod(name))() + 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 _ @@ -2475,7 +2423,7 @@ def _bin_op(name, doc="binary operator"): """ def _(self, other): jc = other._jc if isinstance(other, Column) else other - njc = getattr(self._jc, _scalaMethod(name))(jc) + njc = getattr(self._jc, name)(jc) return Column(njc, self.sql_ctx) _.__doc__ = doc return _ @@ -2486,7 +2434,7 @@ def _reverse_op(name, doc="binary operator"): """ def _(self, other): jother = _create_column_from_literal(other) - jc = getattr(jother, _scalaMethod(name))(self._jc) + jc = getattr(jother, name)(self._jc) return Column(jc, self.sql_ctx) _.__doc__ = doc return _ @@ -2513,34 +2461,33 @@ def __init__(self, jc, sql_ctx=None): super(Column, self).__init__(jc, sql_ctx) # arithmetic operators - __neg__ = _unary_op("unary_-") - __add__ = _bin_op("+") - __sub__ = _bin_op("-") - __mul__ = _bin_op("*") - __div__ = _bin_op("/") - __mod__ = _bin_op("%") - __radd__ = _bin_op("+") - __rsub__ = _reverse_op("-") - __rmul__ = _bin_op("*") - __rdiv__ = _reverse_op("/") - __rmod__ = _reverse_op("%") - __abs__ = _unary_op("abs") + __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("===") - __ne__ = _bin_op("!==") - __lt__ = _bin_op("<") - __le__ = _bin_op("<=") - __ge__ = _bin_op(">=") - __gt__ = _bin_op(">") + __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('&&') - __or__ = _bin_op('||') - __invert__ = _unary_op('unary_!') - __rand__ = _bin_op("&&") - __ror__ = _bin_op("||") + __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") @@ -2582,24 +2529,20 @@ def substr(self, startPos, length): isNull = _unary_op("isNull", "True if the current expression is null.") isNotNull = _unary_op("isNotNull", "True if the current expression is not null.") - # `as` is keyword def alias(self, alias): """Return a alias for this column - >>> df.age.As("age2").collect() - [Row(age2=2), Row(age2=5)] >>> df.age.alias("age2").collect() [Row(age2=2), Row(age2=5)] """ return Column(getattr(self._jc, "as")(alias), self.sql_ctx) - As = alias def cast(self, dataType): """ Convert the column into type `dataType` - >>> df.select(df.age.cast("string").As('ages')).collect() + >>> df.select(df.age.cast("string").alias('ages')).collect() [Row(ages=u'2'), Row(ages=u'5')] - >>> df.select(df.age.cast(StringType()).As('ages')).collect() + >>> df.select(df.age.cast(StringType()).alias('ages')).collect() [Row(ages=u'2'), Row(ages=u'5')] """ if self.sql_ctx is None: @@ -2626,6 +2569,40 @@ def _(col): 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 @@ -2659,7 +2636,7 @@ 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).As('c')).collect() + >>> df.agg(Dsl.countDistinct(df.age, df.name).alias('c')).collect() [Row(c=2)] """ sc = SparkContext._active_spark_context @@ -2674,7 +2651,7 @@ 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).As('c')).collect() + >>> df.agg(Dsl.approxCountDistinct(df.age).alias('c')).collect() [Row(c=2)] """ sc = SparkContext._active_spark_context @@ -2684,6 +2661,16 @@ def approxCountDistinct(col, rsd=None): 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 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 ddce77deb83e1..4c2aeadae9492 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 @@ -128,7 +128,6 @@ trait Column extends DataFrame { */ def unary_! : Column = exprToColumn(Not(expr)) - /** * Equality test. * {{{ @@ -166,13 +165,29 @@ trait Column extends DataFrame { * * // Java: * import static org.apache.spark.sql.Dsl.*; - * df.filter( not(col("colA").equalTo(col("colB"))) ); + * df.filter( col("colA").notEqual(col("colB")) ); * }}} */ def !== (other: Any): Column = constructColumn(other) { o => Not(EqualTo(expr, o.expr)) } + /** + * Inequality test. + * {{{ + * // Scala: + * df.select( df("colA") !== df("colB") ) + * df.select( !(df("colA") === df("colB")) ) + * + * // Java: + * import static org.apache.spark.sql.Dsl.*; + * df.filter( col("colA").notEqual(col("colB")) ); + * }}} + */ + def notEqual(other: Any): Column = constructColumn(other) { o => + Not(EqualTo(expr, o.expr)) + } + /** * Greater than. * {{{ 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 8d7c2a1b8339e..c60d4070942a9 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 @@ -17,7 +17,13 @@ package org.apache.spark.sql +import java.util.{List => JList, Map => JMap} + +import org.apache.spark.Accumulator +import org.apache.spark.api.python.PythonBroadcast +import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.expressions.ScalaUdf +import org.apache.spark.sql.execution.PythonUDF import org.apache.spark.sql.types.DataType /** @@ -37,3 +43,24 @@ case class UserDefinedFunction(f: AnyRef, dataType: DataType) { Column(ScalaUdf(f, dataType, exprs.map(_.expr))) } } + +/** + * A user-defined Python function. To create one, use the `pythonUDF` functions in [[Dsl]]. + * This is used by Python API. + */ +private[sql] case class UserDefinedPythonFunction( + name: String, + command: Array[Byte], + envVars: JMap[String, String], + pythonIncludes: JList[String], + pythonExec: String, + broadcastVars: JList[Broadcast[PythonBroadcast]], + accumulator: Accumulator[JList[Array[Byte]]], + dataType: DataType) { + + def apply(exprs: Column*): Column = { + val udf = PythonUDF(name, command, envVars, pythonIncludes, pythonExec, broadcastVars, + accumulator, dataType, exprs.map(_.expr)) + Column(udf) + } +} From 9a7ce70eabc0ccaa036e142fc97bf0d37faa0b63 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 4 Feb 2015 17:18:03 -0800 Subject: [PATCH 077/578] [SPARK-5411] Allow SparkListeners to be specified in SparkConf and loaded when creating SparkContext This patch introduces a new configuration option, `spark.extraListeners`, that allows SparkListeners to be specified in SparkConf and registered before the SparkContext is initialized. From the configuration documentation: > 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. This motivation for this patch is to allow monitoring code to be easily injected into existing Spark programs without having to modify those programs' code. Author: Josh Rosen Closes #4111 from JoshRosen/SPARK-5190-register-sparklistener-in-sc-constructor and squashes the following commits: 8370839 [Josh Rosen] Two minor fixes after merging with master 6e0122c [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-5190-register-sparklistener-in-sc-constructor 1a5b9a0 [Josh Rosen] Remove SPARK_EXTRA_LISTENERS environment variable. 2daff9b [Josh Rosen] Add a couple of explanatory comments for SPARK_EXTRA_LISTENERS. b9973da [Josh Rosen] Add test to ensure that conf and env var settings are merged, not overriden. d6f3113 [Josh Rosen] Use getConstructors() instead of try-catch to find right constructor. d0d276d [Josh Rosen] Move code into setupAndStartListenerBus() method b22b379 [Josh Rosen] Instantiate SparkListeners from classes listed in configurations. 9c0d8f1 [Josh Rosen] Revert "[SPARK-5190] Allow SparkListeners to be registered before SparkContext starts." 217ecc0 [Josh Rosen] Revert "Add addSparkListener to JavaSparkContext" 25988f3 [Josh Rosen] Add addSparkListener to JavaSparkContext 163ba19 [Josh Rosen] [SPARK-5190] Allow SparkListeners to be registered before SparkContext starts. --- .../scala/org/apache/spark/SparkContext.scala | 57 ++++++++++++++++++- .../org/apache/spark/util/ListenerBus.scala | 3 +- .../spark/scheduler/SparkListenerSuite.scala | 56 ++++++++++++------ docs/configuration.md | 11 ++++ 4 files changed, 106 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 7f5aef1c75df2..a7adddb6c83ec 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -20,6 +20,7 @@ package org.apache.spark import scala.language.implicitConversions import java.io._ +import java.lang.reflect.Constructor import java.net.URI import java.util.{Arrays, Properties, UUID} import java.util.concurrent.atomic.AtomicInteger @@ -387,9 +388,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } executorAllocationManager.foreach(_.start()) - // At this point, all relevant SparkListeners have been registered, so begin releasing events - listenerBus.start() - private[spark] val cleaner: Option[ContextCleaner] = { if (conf.getBoolean("spark.cleaner.referenceTracking", true)) { Some(new ContextCleaner(this)) @@ -399,6 +397,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } cleaner.foreach(_.start()) + setupAndStartListenerBus() postEnvironmentUpdate() postApplicationStart() @@ -1563,6 +1562,58 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli /** Register a new RDD, returning its RDD ID */ private[spark] def newRddId(): Int = nextRddId.getAndIncrement() + /** + * Registers listeners specified in spark.extraListeners, then starts the listener bus. + * This should be called after all internal listeners have been registered with the listener bus + * (e.g. after the web UI and event logging listeners have been registered). + */ + private def setupAndStartListenerBus(): Unit = { + // Use reflection to instantiate listeners specified via `spark.extraListeners` + try { + val listenerClassNames: Seq[String] = + conf.get("spark.extraListeners", "").split(',').map(_.trim).filter(_ != "") + for (className <- listenerClassNames) { + // Use reflection to find the right constructor + val constructors = { + val listenerClass = Class.forName(className) + listenerClass.getConstructors.asInstanceOf[Array[Constructor[_ <: SparkListener]]] + } + val constructorTakingSparkConf = constructors.find { c => + c.getParameterTypes.sameElements(Array(classOf[SparkConf])) + } + lazy val zeroArgumentConstructor = constructors.find { c => + c.getParameterTypes.isEmpty + } + val listener: SparkListener = { + if (constructorTakingSparkConf.isDefined) { + constructorTakingSparkConf.get.newInstance(conf) + } else if (zeroArgumentConstructor.isDefined) { + zeroArgumentConstructor.get.newInstance() + } else { + throw new SparkException( + s"$className did not have a zero-argument constructor or a" + + " single-argument constructor that accepts SparkConf. Note: if the class is" + + " defined inside of another Scala class, then its constructors may accept an" + + " implicit parameter that references the enclosing class; in this case, you must" + + " define the listener as a top-level class in order to prevent this extra" + + " parameter from breaking Spark's ability to find a valid constructor.") + } + } + listenerBus.addListener(listener) + logInfo(s"Registered listener $className") + } + } catch { + case e: Exception => + try { + stop() + } finally { + throw new SparkException(s"Exception when registering SparkListener", e) + } + } + + listenerBus.start() + } + /** Post the application start event */ private def postApplicationStart() { // Note: this code assumes that the task scheduler has been initialized and has contacted 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 bd0aa4dc4650f..d60b8b9a31a9b 100644 --- a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala @@ -28,7 +28,8 @@ import org.apache.spark.Logging */ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { - private val listeners = new CopyOnWriteArrayList[L] + // Marked `private[spark]` for access in tests. + private[spark] val listeners = new CopyOnWriteArrayList[L] /** * Add a listener to listen events. This method is thread-safe and can be called in any thread. 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 0fb1bdd30d975..3a41ee8d4ae0c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -20,26 +20,22 @@ package org.apache.spark.scheduler import java.util.concurrent.Semaphore import scala.collection.mutable +import scala.collection.JavaConversions._ -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} -import org.scalatest.Matchers +import org.scalatest.{FunSuite, Matchers} -import org.apache.spark.{LocalSparkContext, SparkContext} import org.apache.spark.executor.TaskMetrics import org.apache.spark.util.ResetSystemProperties +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext} -class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers with BeforeAndAfter - with BeforeAndAfterAll with ResetSystemProperties { +class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers + with ResetSystemProperties { /** Length of time to wait while draining listener events. */ val WAIT_TIMEOUT_MILLIS = 10000 val jobCompletionTime = 1421191296660L - before { - sc = new SparkContext("local", "SparkListenerSuite") - } - test("basic creation and shutdown of LiveListenerBus") { val counter = new BasicJobCounter val bus = new LiveListenerBus @@ -127,6 +123,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers } test("basic creation of StageInfo") { + sc = new SparkContext("local", "SparkListenerSuite") val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) val rdd1 = sc.parallelize(1 to 100, 4) @@ -148,6 +145,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers } test("basic creation of StageInfo with shuffle") { + sc = new SparkContext("local", "SparkListenerSuite") val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) val rdd1 = sc.parallelize(1 to 100, 4) @@ -185,6 +183,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers } test("StageInfo with fewer tasks than partitions") { + sc = new SparkContext("local", "SparkListenerSuite") val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) val rdd1 = sc.parallelize(1 to 100, 4) @@ -201,6 +200,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers } test("local metrics") { + sc = new SparkContext("local", "SparkListenerSuite") val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) sc.addSparkListener(new StatsReportListener) @@ -267,6 +267,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers } test("onTaskGettingResult() called when result fetched remotely") { + sc = new SparkContext("local", "SparkListenerSuite") val listener = new SaveTaskEvents sc.addSparkListener(listener) @@ -287,6 +288,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers } test("onTaskGettingResult() not called when result sent directly") { + sc = new SparkContext("local", "SparkListenerSuite") val listener = new SaveTaskEvents sc.addSparkListener(listener) @@ -302,6 +304,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers } test("onTaskEnd() should be called for all started tasks, even after job has been killed") { + sc = new SparkContext("local", "SparkListenerSuite") val WAIT_TIMEOUT_MILLIS = 10000 val listener = new SaveTaskEvents sc.addSparkListener(listener) @@ -356,6 +359,17 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers assert(jobCounter2.count === 5) } + test("registering listeners via spark.extraListeners") { + val conf = new SparkConf().setMaster("local").setAppName("test") + .set("spark.extraListeners", classOf[ListenerThatAcceptsSparkConf].getName + "," + + classOf[BasicJobCounter].getName) + sc = new SparkContext(conf) + sc.listenerBus.listeners.collect { case x: BasicJobCounter => x}.size should be (1) + sc.listenerBus.listeners.collect { + case x: ListenerThatAcceptsSparkConf => x + }.size should be (1) + } + /** * Assert that the given list of numbers has an average that is greater than zero. */ @@ -363,14 +377,6 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers assert(m.sum / m.size.toDouble > 0.0, msg) } - /** - * A simple listener that counts the number of jobs observed. - */ - private class BasicJobCounter extends SparkListener { - var count = 0 - override def onJobEnd(job: SparkListenerJobEnd) = count += 1 - } - /** * A simple listener that saves all task infos and task metrics. */ @@ -423,3 +429,19 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers } } + +// These classes can't be declared inside of the SparkListenerSuite class because we don't want +// their constructors to contain references to SparkListenerSuite: + +/** + * A simple listener that counts the number of jobs observed. + */ +private class BasicJobCounter extends SparkListener { + var count = 0 + override def onJobEnd(job: SparkListenerJobEnd) = count += 1 +} + +private class ListenerThatAcceptsSparkConf(conf: SparkConf) extends SparkListener { + var count = 0 + override def onJobEnd(job: SparkListenerJobEnd) = count += 1 +} diff --git a/docs/configuration.md b/docs/configuration.md index 62d3fca937b2c..8b1d7598c47e4 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -190,6 +190,17 @@ of the most common options to set are: Logs the effective SparkConf as INFO when a SparkContext is started. + + + + +
Property NameDefaultMeaning
spark.sql.parquet.int96AsTimestamptrue + Some Parquet-producing systems, in particular Impala, store Timestamp into INT96. Spark would also + store Timestamp as INT96 because we need to avoid precision lost of the nanoseconds field. This + flag tells Spark SQL to interpret INT96 data as a timestamp to provide compatibility with these systems. +
spark.sql.parquet.cacheMetadata true
spark.extraListeners(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. +
Apart from these, the following properties are also available, and may be useful in some situations: From 1fbd124b1bd6159086d8e88b139ce0817af02322 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 4 Feb 2015 18:35:51 -0800 Subject: [PATCH 078/578] [SPARK-5605][SQL][DF] Allow using String to specify colum name in DSL aggregate functions Author: Reynold Xin Closes #4376 from rxin/SPARK-5605 and squashes the following commits: c55f5fa [Reynold Xin] Added a Python test. f4b8dbb [Reynold Xin] [SPARK-5605][SQL][DF] Allow using String to specify colum name in DSL aggregate functions. --- python/pyspark/sql.py | 13 ++++--- .../org/apache/spark/sql/DataFrame.scala | 8 ++-- .../org/apache/spark/sql/DataFrameImpl.scala | 8 ++-- .../main/scala/org/apache/spark/sql/Dsl.scala | 37 +++++++++++++++++++ ...oupedDataFrame.scala => GroupedData.scala} | 2 +- .../apache/spark/sql/IncomputableColumn.scala | 4 +- 6 files changed, 56 insertions(+), 16 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/{GroupedDataFrame.scala => GroupedData.scala} (98%) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 5b56b36bdcdb7..417db34d679fc 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -23,7 +23,7 @@ - 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{GroupedDataFrame} + - L{GroupedData} - L{Column} Column is a DataFrame with a single column. - L{Row} @@ -62,7 +62,7 @@ "StringType", "BinaryType", "BooleanType", "DateType", "TimestampType", "DecimalType", "DoubleType", "FloatType", "ByteType", "IntegerType", "LongType", "ShortType", "ArrayType", "MapType", "StructField", "StructType", - "SQLContext", "HiveContext", "DataFrame", "GroupedDataFrame", "Column", "Row", "Dsl", + "SQLContext", "HiveContext", "DataFrame", "GroupedData", "Column", "Row", "Dsl", "SchemaRDD"] @@ -2231,7 +2231,7 @@ def filter(self, condition): def groupBy(self, *cols): """ Group the :class:`DataFrame` using the specified columns, - so we can run aggregation on them. See :class:`GroupedDataFrame` + so we can run aggregation on them. See :class:`GroupedData` for all the available aggregate functions. >>> df.groupBy().avg().collect() @@ -2244,7 +2244,7 @@ def groupBy(self, *cols): 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 GroupedDataFrame(jdf, self.sql_ctx) + return GroupedData(jdf, self.sql_ctx) def agg(self, *exprs): """ Aggregate on the entire :class:`DataFrame` without groups @@ -2308,7 +2308,7 @@ def _api(self): return _api -class GroupedDataFrame(object): +class GroupedData(object): """ A set of methods for aggregations on a :class:`DataFrame`, @@ -2638,6 +2638,9 @@ def countDistinct(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], 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 a4997fb293781..92e04ce17c2e7 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 @@ -290,7 +290,7 @@ trait DataFrame extends RDDApi[Row] { /** * Groups the [[DataFrame]] using the specified columns, so we can run aggregation on them. - * See [[GroupedDataFrame]] for all the available aggregate functions. + * See [[GroupedData]] for all the available aggregate functions. * * {{{ * // Compute the average for all numeric columns grouped by department. @@ -304,11 +304,11 @@ trait DataFrame extends RDDApi[Row] { * }}} */ @scala.annotation.varargs - def groupBy(cols: Column*): GroupedDataFrame + def groupBy(cols: Column*): GroupedData /** * Groups the [[DataFrame]] using the specified columns, so we can run aggregation on them. - * See [[GroupedDataFrame]] for all the available aggregate functions. + * See [[GroupedData]] for all the available aggregate functions. * * This is a variant of groupBy that can only group by existing columns using column names * (i.e. cannot construct expressions). @@ -325,7 +325,7 @@ trait DataFrame extends RDDApi[Row] { * }}} */ @scala.annotation.varargs - def groupBy(col1: String, cols: String*): GroupedDataFrame + def groupBy(col1: String, cols: String*): GroupedData /** * (Scala-specific) Compute aggregates by specifying a map from column name to 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 c702adcb65122..d6df927f9d42c 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,13 @@ private[sql] class DataFrameImpl protected[sql]( filter(condition) } - override def groupBy(cols: Column*): GroupedDataFrame = { - new GroupedDataFrame(this, cols.map(_.expr)) + override def groupBy(cols: Column*): GroupedData = { + new GroupedData(this, cols.map(_.expr)) } - override def groupBy(col1: String, cols: String*): GroupedDataFrame = { + override def groupBy(col1: String, cols: String*): GroupedData = { val colNames: Seq[String] = col1 +: cols - new GroupedDataFrame(this, colNames.map(colName => resolve(colName))) + new GroupedData(this, colNames.map(colName => resolve(colName))) } override def limit(n: Int): 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 50f442dd87bf3..9afe496edc2be 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 @@ -94,38 +94,75 @@ object Dsl { /** Aggregate function: returns the sum of all values in the expression. */ def sum(e: Column): Column = Sum(e.expr) + /** Aggregate function: returns the sum of all values in the given column. */ + def sum(columnName: String): Column = sum(Column(columnName)) + /** Aggregate function: returns the sum of distinct values in the expression. */ def sumDistinct(e: Column): Column = SumDistinct(e.expr) + /** Aggregate function: returns the sum of distinct values in the expression. */ + 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) + /** Aggregate function: returns the number of items in a group. */ + def count(columnName: String): Column = count(Column(columnName)) + /** Aggregate function: returns the number of distinct items in a group. */ @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. */ + @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. */ def approxCountDistinct(e: Column): Column = ApproxCountDistinct(e.expr) + /** Aggregate function: returns the approximate number of distinct items in a group. */ + def approxCountDistinct(columnName: String): Column = approxCountDistinct(column(columnName)) + /** Aggregate function: returns the approximate number of distinct items in a group. */ def approxCountDistinct(e: Column, rsd: Double): Column = ApproxCountDistinct(e.expr, rsd) + /** Aggregate function: returns the approximate number of distinct items in a group. */ + def approxCountDistinct(columnName: String, rsd: Double): Column = { + approxCountDistinct(Column(columnName), rsd) + } + /** Aggregate function: returns the average of the values in a group. */ def avg(e: Column): Column = Average(e.expr) + /** Aggregate function: returns the average of the values in a group. */ + def avg(columnName: String): Column = avg(Column(columnName)) + /** Aggregate function: returns the first value in a group. */ def first(e: Column): Column = First(e.expr) + /** Aggregate function: returns the first value of a column in a group. */ + def first(columnName: String): Column = first(Column(columnName)) + /** Aggregate function: returns the last value in a group. */ def last(e: Column): Column = Last(e.expr) + /** Aggregate function: returns the last value of the column in a group. */ + def last(columnName: String): Column = last(Column(columnName)) + /** Aggregate function: returns the minimum value of the expression in a group. */ def min(e: Column): Column = Min(e.expr) + /** Aggregate function: returns the minimum value of the column in a group. */ + def min(columnName: String): Column = min(Column(columnName)) + /** Aggregate function: returns the maximum value of the expression in a group. */ def max(e: Column): Column = Max(e.expr) + /** Aggregate function: returns the maximum value of the column in a group. */ + def max(columnName: String): Column = max(Column(columnName)) + ////////////////////////////////////////////////////////////////////////////////////////////// ////////////////////////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala rename to sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala index 7963cb03126ba..3c20676355c9d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical.Aggregate /** * A set of methods for aggregations on a [[DataFrame]], created by [[DataFrame.groupBy]]. */ -class GroupedDataFrame protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expression]) { +class GroupedData protected[sql](df: DataFrameImpl, groupingExprs: Seq[Expression]) { private[this] implicit def toDataFrame(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 index 6b032d3d699a9..fedd7f06ef50a 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 @@ -90,9 +90,9 @@ private[sql] class IncomputableColumn(protected[sql] val expr: Expression) exten override def apply(condition: Column): DataFrame = err() - override def groupBy(cols: Column*): GroupedDataFrame = err() + override def groupBy(cols: Column*): GroupedData = err() - override def groupBy(col1: String, cols: String*): GroupedDataFrame = err() + override def groupBy(col1: String, cols: String*): GroupedData = err() override def limit(n: Int): DataFrame = err() From dba98bf6987ec39380f1a5b0ca2772b694452231 Mon Sep 17 00:00:00 2001 From: Sadhan Sood Date: Wed, 4 Feb 2015 19:18:06 -0800 Subject: [PATCH 079/578] [SPARK-4520] [SQL] This pr fixes the ArrayIndexOutOfBoundsException as r... ...aised in SPARK-4520. The exception is thrown only for a thrift generated parquet file. The array element schema name is assumed as "array" as per ParquetAvro but for thrift generated parquet files, it is array_name + "_tuple". This leads to missing child of array group type and hence when the parquet rows are being materialized leads to the exception. Author: Sadhan Sood Closes #4148 from sadhan/SPARK-4520 and squashes the following commits: c5ccde8 [Sadhan Sood] [SPARK-4520] [SQL] This pr fixes the ArrayIndexOutOfBoundsException as raised in SPARK-4520. --- .../spark/sql/parquet/ParquetConverter.scala | 5 +++ .../sql/parquet/ParquetTableSupport.scala | 6 +++- .../spark/sql/parquet/ParquetTypes.scala | 35 +++++++++++++------ .../sql/parquet/ParquetSchemaSuite.scala | 28 +++++++++++++-- 4 files changed, 60 insertions(+), 14 deletions(-) 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 d87ddfeabda77..7d62f3728f036 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 @@ -66,6 +66,11 @@ private[sql] object CatalystConverter { // Using a different value will result in Parquet silently dropping columns. val ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME = "bag" val ARRAY_ELEMENTS_SCHEMA_NAME = "array" + // SPARK-4520: Thrift generated parquet files have different array element + // schema names than avro. Thrift parquet uses array_schema_name + "_tuple" + // as opposed to "array" used by default. For more information, check + // TestThriftSchemaConverter.java in parquet.thrift. + val THRIFT_ARRAY_ELEMENTS_SCHEMA_NAME_SUFFIX = "_tuple" val MAP_KEY_SCHEMA_NAME = "key" val MAP_VALUE_SCHEMA_NAME = "value" val MAP_SCHEMA_NAME = "map" 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 3fb1cc410521e..14c81ae4eba4e 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 @@ -99,7 +99,11 @@ private[parquet] class RowReadSupport extends ReadSupport[Row] with Logging { val requestedAttributes = RowReadSupport.getRequestedSchema(configuration) if (requestedAttributes != null) { - parquetSchema = ParquetTypesConverter.convertFromAttributes(requestedAttributes) + // If the parquet file is thrift derived, there is a good chance that + // it will have the thrift class in metadata. + val isThriftDerived = keyValueMetaData.keySet().contains("thrift.class") + parquetSchema = ParquetTypesConverter + .convertFromAttributes(requestedAttributes, isThriftDerived) metadata.put( RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA, ParquetTypesConverter.convertToString(requestedAttributes)) 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 f1d4ff2387709..b646109b7c553 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 @@ -285,13 +285,19 @@ private[parquet] object ParquetTypesConverter extends Logging { ctype: DataType, name: String, nullable: Boolean = true, - inArray: Boolean = false): ParquetType = { + inArray: Boolean = false, + toThriftSchemaNames: Boolean = false): ParquetType = { val repetition = if (inArray) { Repetition.REPEATED } else { if (nullable) Repetition.OPTIONAL else Repetition.REQUIRED } + val arraySchemaName = if (toThriftSchemaNames) { + name + CatalystConverter.THRIFT_ARRAY_ELEMENTS_SCHEMA_NAME_SUFFIX + } else { + CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME + } val typeInfo = fromPrimitiveDataType(ctype) typeInfo.map { case ParquetTypeInfo(primitiveType, originalType, decimalMetadata, length) => @@ -306,22 +312,24 @@ private[parquet] object ParquetTypesConverter extends Logging { }.getOrElse { ctype match { case udt: UserDefinedType[_] => { - fromDataType(udt.sqlType, name, nullable, inArray) + fromDataType(udt.sqlType, name, nullable, inArray, toThriftSchemaNames) } case ArrayType(elementType, false) => { val parquetElementType = fromDataType( elementType, - CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, + arraySchemaName, nullable = false, - inArray = true) + inArray = true, + toThriftSchemaNames) ConversionPatterns.listType(repetition, name, parquetElementType) } case ArrayType(elementType, true) => { val parquetElementType = fromDataType( elementType, - CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, + arraySchemaName, nullable = true, - inArray = false) + inArray = false, + toThriftSchemaNames) ConversionPatterns.listType( repetition, name, @@ -332,7 +340,8 @@ private[parquet] object ParquetTypesConverter extends Logging { } case StructType(structFields) => { val fields = structFields.map { - field => fromDataType(field.dataType, field.name, field.nullable, inArray = false) + field => fromDataType(field.dataType, field.name, field.nullable, + inArray = false, toThriftSchemaNames) } new ParquetGroupType(repetition, name, fields.toSeq) } @@ -342,13 +351,15 @@ private[parquet] object ParquetTypesConverter extends Logging { keyType, CatalystConverter.MAP_KEY_SCHEMA_NAME, nullable = false, - inArray = false) + inArray = false, + toThriftSchemaNames) val parquetValueType = fromDataType( valueType, CatalystConverter.MAP_VALUE_SCHEMA_NAME, nullable = valueContainsNull, - inArray = false) + inArray = false, + toThriftSchemaNames) ConversionPatterns.mapType( repetition, name, @@ -374,10 +385,12 @@ private[parquet] object ParquetTypesConverter extends Logging { field.getRepetition != Repetition.REQUIRED)()) } - def convertFromAttributes(attributes: Seq[Attribute]): MessageType = { + def convertFromAttributes(attributes: Seq[Attribute], + toThriftSchemaNames: Boolean = false): MessageType = { val fields = attributes.map( attribute => - fromDataType(attribute.dataType, attribute.name, attribute.nullable)) + fromDataType(attribute.dataType, attribute.name, attribute.nullable, + toThriftSchemaNames = toThriftSchemaNames)) new MessageType("root", fields) } 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 64274950b868e..5f7f31d395cf7 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 @@ -33,9 +33,10 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest { * Checks whether the reflected Parquet message type for product type `T` conforms `messageType`. */ private def testSchema[T <: Product: ClassTag: TypeTag]( - testName: String, messageType: String): Unit = { + testName: String, messageType: String, isThriftDerived: Boolean = false): Unit = { test(testName) { - val actual = ParquetTypesConverter.convertFromAttributes(ScalaReflection.attributesFor[T]) + val actual = ParquetTypesConverter.convertFromAttributes(ScalaReflection.attributesFor[T], + isThriftDerived) val expected = MessageTypeParser.parseMessageType(messageType) actual.checkContains(expected) expected.checkContains(actual) @@ -146,6 +147,29 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest { |} """.stripMargin) + // Test for SPARK-4520 -- ensure that thrift generated parquet schema is generated + // as expected from attributes + testSchema[(Array[Byte], Array[Byte], Array[Byte], Seq[Int], Map[Array[Byte], Seq[Int]])]( + "thrift generated parquet schema", + """ + |message root { + | optional binary _1 (UTF8); + | optional binary _2 (UTF8); + | optional binary _3 (UTF8); + | optional group _4 (LIST) { + | repeated int32 _4_tuple; + | } + | optional group _5 (MAP) { + | repeated group map (MAP_KEY_VALUE) { + | required binary key (UTF8); + | optional group value (LIST) { + | repeated int32 value_tuple; + | } + | } + | } + |} + """.stripMargin, isThriftDerived = true) + test("DataType string parser compatibility") { // This is the generated string from previous versions of the Spark SQL, using the following: // val schema = StructType(List( From 6b4c7f08068b6099145ab039d0499e3fef68e2e9 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 4 Feb 2015 19:51:48 -0800 Subject: [PATCH 080/578] [SQL][DataFrame] Minor cleanup. 1. Removed LocalHiveContext in Python. 2. Reduced DSL UDF support from 22 arguments to 10 arguments so JavaDoc/ScalaDoc look nicer. Author: Reynold Xin Closes #4374 from rxin/df-style and squashes the following commits: e493342 [Reynold Xin] [SQL][DataFrame] Minor cleanup. --- python/pyspark/sql.py | 11 - .../main/scala/org/apache/spark/sql/Dsl.scala | 196 +----------------- 2 files changed, 2 insertions(+), 205 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 417db34d679fc..3ac8ea597e142 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1683,17 +1683,6 @@ def _get_hive_ctx(self): return self._jvm.HiveContext(self._jsc.sc()) -class LocalHiveContext(HiveContext): - - def __init__(self, sparkContext, sqlContext=None): - HiveContext.__init__(self, sparkContext, sqlContext) - warnings.warn("LocalHiveContext is deprecated. " - "Use HiveContext instead.", DeprecationWarning) - - def _get_hive_ctx(self): - return self._jvm.LocalHiveContext(self._jsc.sc()) - - def _create_row(fields, values): row = Row(*values) row.__FIELDS__ = fields 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 9afe496edc2be..6bf21dd1bc79b 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 @@ -209,7 +209,7 @@ object Dsl { // scalastyle:off /* Use the following code to generate: - (0 to 22).map { x => + (0 to 10).map { x => val types = (1 to x).foldRight("RT")((i, s) => {s"A$i, $s"}) val typeTags = (1 to x).map(i => s"A$i: TypeTag").foldLeft("RT: TypeTag")(_ + ", " + _) println(s""" @@ -222,7 +222,7 @@ object Dsl { }""") } - (0 to 22).map { x => + (0 to 10).map { x => val args = (1 to x).map(i => s"arg$i: Column").mkString(", ") val fTypes = Seq.fill(x + 1)("_").mkString(", ") val argsInUdf = (1 to x).map(i => s"arg$i.expr").mkString(", ") @@ -325,102 +325,6 @@ object Dsl { UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) } - /** - * Defines a user-defined function of 11 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. - */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag](f: Function11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, RT]): UserDefinedFunction = { - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) - } - - /** - * Defines a user-defined function of 12 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. - */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag](f: Function12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, RT]): UserDefinedFunction = { - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) - } - - /** - * Defines a user-defined function of 13 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. - */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag](f: Function13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, RT]): UserDefinedFunction = { - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) - } - - /** - * Defines a user-defined function of 14 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. - */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag](f: Function14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, RT]): UserDefinedFunction = { - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) - } - - /** - * Defines a user-defined function of 15 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. - */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag](f: Function15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, RT]): UserDefinedFunction = { - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) - } - - /** - * Defines a user-defined function of 16 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. - */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag](f: Function16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, RT]): UserDefinedFunction = { - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) - } - - /** - * Defines a user-defined function of 17 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. - */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag](f: Function17[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, RT]): UserDefinedFunction = { - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) - } - - /** - * Defines a user-defined function of 18 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. - */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag](f: Function18[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, RT]): UserDefinedFunction = { - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) - } - - /** - * Defines a user-defined function of 19 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. - */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag](f: Function19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, RT]): UserDefinedFunction = { - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) - } - - /** - * Defines a user-defined function of 20 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. - */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag](f: Function20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, RT]): UserDefinedFunction = { - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) - } - - /** - * Defines a user-defined function of 21 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. - */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag](f: Function21[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, RT]): UserDefinedFunction = { - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) - } - - /** - * Defines a user-defined function of 22 arguments as user-defined function (UDF). - * The data types are automatically inferred based on the function's signature. - */ - def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag, A22: TypeTag](f: Function22[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, RT]): UserDefinedFunction = { - UserDefinedFunction(f, ScalaReflection.schemaFor(typeTag[RT]).dataType) - } - ////////////////////////////////////////////////////////////////////////////////////////////////// /** @@ -511,101 +415,5 @@ object Dsl { ScalaUdf(f, returnType, Seq(arg1.expr, arg2.expr, arg3.expr, arg4.expr, arg5.expr, arg6.expr, arg7.expr, arg8.expr, arg9.expr, arg10.expr)) } - /** - * Call a Scala function of 11 arguments as user-defined function (UDF). This requires - * you to specify the return data type. - */ - def callUDF(f: Function11[_, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: 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, arg11.expr)) - } - - /** - * Call a Scala function of 12 arguments as user-defined function (UDF). This requires - * you to specify the return data type. - */ - def callUDF(f: Function12[_, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: 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, arg11.expr, arg12.expr)) - } - - /** - * Call a Scala function of 13 arguments as user-defined function (UDF). This requires - * you to specify the return data type. - */ - def callUDF(f: Function13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: 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, arg11.expr, arg12.expr, arg13.expr)) - } - - /** - * Call a Scala function of 14 arguments as user-defined function (UDF). This requires - * you to specify the return data type. - */ - def callUDF(f: Function14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: 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, arg11.expr, arg12.expr, arg13.expr, arg14.expr)) - } - - /** - * Call a Scala function of 15 arguments as user-defined function (UDF). This requires - * you to specify the return data type. - */ - def callUDF(f: Function15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: 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, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr)) - } - - /** - * Call a Scala function of 16 arguments as user-defined function (UDF). This requires - * you to specify the return data type. - */ - def callUDF(f: Function16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: 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, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr)) - } - - /** - * Call a Scala function of 17 arguments as user-defined function (UDF). This requires - * you to specify the return data type. - */ - def callUDF(f: Function17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: 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, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr)) - } - - /** - * Call a Scala function of 18 arguments as user-defined function (UDF). This requires - * you to specify the return data type. - */ - def callUDF(f: Function18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: 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, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr)) - } - - /** - * Call a Scala function of 19 arguments as user-defined function (UDF). This requires - * you to specify the return data type. - */ - def callUDF(f: Function19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: 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, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr)) - } - - /** - * Call a Scala function of 20 arguments as user-defined function (UDF). This requires - * you to specify the return data type. - */ - def callUDF(f: Function20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column, arg20: 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, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr, arg20.expr)) - } - - /** - * Call a Scala function of 21 arguments as user-defined function (UDF). This requires - * you to specify the return data type. - */ - def callUDF(f: Function21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column, arg20: Column, arg21: 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, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr, arg20.expr, arg21.expr)) - } - - /** - * Call a Scala function of 22 arguments as user-defined function (UDF). This requires - * you to specify the return data type. - */ - def callUDF(f: Function22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType, arg1: Column, arg2: Column, arg3: Column, arg4: Column, arg5: Column, arg6: Column, arg7: Column, arg8: Column, arg9: Column, arg10: Column, arg11: Column, arg12: Column, arg13: Column, arg14: Column, arg15: Column, arg16: Column, arg17: Column, arg18: Column, arg19: Column, arg20: Column, arg21: Column, arg22: 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, arg11.expr, arg12.expr, arg13.expr, arg14.expr, arg15.expr, arg16.expr, arg17.expr, arg18.expr, arg19.expr, arg20.expr, arg21.expr, arg22.expr)) - } - // scalastyle:on } From 206f9bc3622348926d73e43c8010519f7df9b34f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 4 Feb 2015 19:52:41 -0800 Subject: [PATCH 081/578] [SPARK-5538][SQL] Fix flaky CachedTableSuite Author: Reynold Xin Closes #4379 from rxin/CachedTableSuite and squashes the following commits: f2b44ce [Reynold Xin] [SQL] Fix flaky CachedTableSuite. --- .../apache/spark/sql/CachedTableSuite.scala | 23 +++++++++++++++---- 1 file changed, 19 insertions(+), 4 deletions(-) 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 c9221f8f934ad..acb5677c4b77b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -17,6 +17,12 @@ package org.apache.spark.sql +import scala.concurrent.duration._ +import scala.language.implicitConversions +import scala.language.postfixOps + +import org.scalatest.concurrent.Eventually._ + import org.apache.spark.sql.TestData._ import org.apache.spark.sql.columnar._ import org.apache.spark.sql.Dsl._ @@ -191,7 +197,10 @@ class CachedTableSuite extends QueryTest { sql("UNCACHE TABLE testData") assert(!isCached("testData"), "Table 'testData' should not be cached") - assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + + eventually(timeout(10 seconds)) { + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } } test("CACHE TABLE tableName AS SELECT * FROM anotherTable") { @@ -204,7 +213,9 @@ class CachedTableSuite extends QueryTest { "Eagerly cached in-memory table should have already been materialized") uncacheTable("testCacheTable") - assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + eventually(timeout(10 seconds)) { + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } } test("CACHE TABLE tableName AS SELECT ...") { @@ -217,7 +228,9 @@ class CachedTableSuite extends QueryTest { "Eagerly cached in-memory table should have already been materialized") uncacheTable("testCacheTable") - assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + eventually(timeout(10 seconds)) { + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } } test("CACHE LAZY TABLE tableName") { @@ -235,7 +248,9 @@ class CachedTableSuite extends QueryTest { "Lazily cached in-memory table should have been materialized") uncacheTable("testData") - assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + eventually(timeout(10 seconds)) { + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } } test("InMemoryRelation statistics") { From 84acd08e0886aa23195f35837c15c09aa7804aff Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 4 Feb 2015 19:53:57 -0800 Subject: [PATCH 082/578] [SPARK-5602][SQL] Better support for creating DataFrame from local data collection 1. Added methods to create DataFrames from Seq[Product] 2. Added executeTake to avoid running a Spark job on LocalRelations. Author: Reynold Xin Closes #4372 from rxin/localDataFrame and squashes the following commits: f696858 [Reynold Xin] style checker. 839ef7f [Reynold Xin] [SPARK-5602][SQL] Better support for creating DataFrame from local data collection. --- .../spark/sql/catalyst/ScalaReflection.scala | 2 +- ...TestRelation.scala => LocalRelation.scala} | 23 +++++---- .../spark/sql/types/DataTypeConversions.scala | 16 ++++++ .../org/apache/spark/sql/SQLContext.scala | 41 ++++++++++++++-- .../spark/sql/execution/ExistingRDD.scala | 36 +++++--------- .../spark/sql/execution/LocalTableScan.scala | 36 ++++++++++++++ .../spark/sql/execution/SparkPlan.scala | 49 ++++++++++++++++++- .../spark/sql/execution/SparkStrategies.scala | 9 +--- .../spark/sql/execution/basicOperators.scala | 44 +---------------- .../apache/spark/sql/execution/commands.scala | 2 + 10 files changed, 170 insertions(+), 88 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/{TestRelation.scala => LocalRelation.scala} (73%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala 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 8e79e532ca564..0445f3aa07912 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 @@ -211,7 +211,7 @@ trait ScalaReflection { */ def asRelation: LocalRelation = { val output = attributesFor[A] - LocalRelation(output, data) + LocalRelation.fromProduct(output, data) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala similarity index 73% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala index d90af45b375e4..92bd057c6f4b6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala @@ -17,31 +17,34 @@ 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.expressions.Attribute -import org.apache.spark.sql.types.{StructType, StructField} +import org.apache.spark.sql.types.{DataTypeConversions, StructType, StructField} object LocalRelation { def apply(output: Attribute*): LocalRelation = new LocalRelation(output) - def apply(output1: StructField, output: StructField*): LocalRelation = new LocalRelation( - StructType(output1 +: output).toAttributes - ) + def apply(output1: StructField, output: StructField*): LocalRelation = { + new LocalRelation(StructType(output1 +: output).toAttributes) + } + + def fromProduct(output: Seq[Attribute], data: Seq[Product]): LocalRelation = { + val schema = StructType.fromAttributes(output) + LocalRelation(output, data.map(row => DataTypeConversions.productToRow(row, schema))) + } } -case class LocalRelation(output: Seq[Attribute], data: Seq[Product] = Nil) +case class LocalRelation(output: Seq[Attribute], data: Seq[Row] = Nil) extends LeafNode with analysis.MultiInstanceRelation { - // TODO: Validate schema compliance. - def loadData(newData: Seq[Product]) = new LocalRelation(output, data ++ newData) - /** * Returns an identical copy of this relation with new exprIds for all attributes. Different * attributes are required when a relation is going to be included multiple times in the same * query. */ - override final def newInstance: this.type = { - LocalRelation(output.map(_.newInstance), data).asInstanceOf[this.type] + override final def newInstance(): this.type = { + LocalRelation(output.map(_.newInstance()), data).asInstanceOf[this.type] } override protected def stringArgs = Iterator(output) 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 21f478c80c94b..c243be07a91b6 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 @@ -19,11 +19,27 @@ 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 protected[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 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 1661282fc31c4..5ab5494f8042f 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 @@ -33,7 +33,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.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution._ import org.apache.spark.sql.json._ @@ -163,17 +163,52 @@ class SQLContext(@transient val sparkContext: SparkContext) /** Removes the specified table from the in-memory cache. */ def uncacheTable(tableName: String): Unit = cacheManager.uncacheTable(tableName) + // scalastyle:off + // Disable style checker so "implicits" object can start with lowercase i + /** + * Implicit methods available in Scala for converting common Scala objects into [[DataFrame]]s. + */ + object implicits { + // scalastyle:on + /** + * Creates a DataFrame from an RDD of case classes. + * + * @group userf + */ + implicit def createDataFrame[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 = { + self.createDataFrame(data) + } + } + /** * Creates a DataFrame from an RDD of case classes. * * @group userf */ - implicit def createDataFrame[A <: Product: TypeTag](rdd: RDD[A]): DataFrame = { + // TODO: Remove implicit here. + implicit def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { SparkPlan.currentContext.set(self) val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] val attributeSeq = schema.toAttributes val rowRDD = RDDConversions.productToRowRdd(rdd, schema) - DataFrame(this, LogicalRDD(attributeSeq, rowRDD)(self)) + DataFrame(self, LogicalRDD(attributeSeq, rowRDD)(self)) + } + + /** + * Creates a DataFrame from a local Seq of Product. + */ + def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { + SparkPlan.currentContext.set(self) + val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] + val attributeSeq = schema.toAttributes + DataFrame(self, LocalRelation.fromProduct(attributeSeq, data)) } /** 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 20b14834bb0d5..248dc1512b4d3 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 @@ -54,12 +54,13 @@ object RDDConversions { } } +/** Logical plan node for scanning data from an RDD. */ case class LogicalRDD(output: Seq[Attribute], rdd: RDD[Row])(sqlContext: SQLContext) extends LogicalPlan with MultiInstanceRelation { - def children = Nil + override def children = Nil - def newInstance() = + override def newInstance() = LogicalRDD(output.map(_.newInstance()), rdd)(sqlContext).asInstanceOf[this.type] override def sameResult(plan: LogicalPlan) = plan match { @@ -74,39 +75,28 @@ 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 } -@deprecated("Use LogicalRDD", "1.2.0") -case class ExistingRdd(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { - override def execute() = rdd -} - -@deprecated("Use LogicalRDD", "1.2.0") -case class SparkLogicalPlan(alreadyPlanned: SparkPlan)(@transient sqlContext: SQLContext) - extends LogicalPlan with MultiInstanceRelation { +/** 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 { - def output = alreadyPlanned.output override def children = Nil - override final def newInstance(): this.type = { - SparkLogicalPlan( - alreadyPlanned match { - case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance()), rdd) - case _ => sys.error("Multiple instance of the same relation detected.") - })(sqlContext).asInstanceOf[this.type] - } + override def newInstance() = + LogicalLocalTable(output.map(_.newInstance()), rows)(sqlContext).asInstanceOf[this.type] override def sameResult(plan: LogicalPlan) = plan match { - case SparkLogicalPlan(ExistingRdd(_, rdd)) => - rdd.id == alreadyPlanned.asInstanceOf[ExistingRdd].rdd.id + case LogicalRDD(_, otherRDD) => rows == rows case _ => false } @transient override lazy val 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) + // 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/LocalTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala new file mode 100644 index 0000000000000..d6d8258f46a9a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.Attribute + + +/** + * Physical plan node for scanning data from a local collection. + */ +case class LocalTableScan(output: Seq[Attribute], rows: Seq[Row]) extends LeafNode { + + private lazy val rdd = sqlContext.sparkContext.parallelize(rows) + + override def execute() = rdd + + override def executeCollect() = rows.toArray + + override def executeTake(limit: Int) = rows.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 6fecd1ff066c3..052766c20abc2 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 @@ -27,6 +27,8 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical._ +import scala.collection.mutable.ArrayBuffer + object SparkPlan { protected[sql] val currentContext = new ThreadLocal[SQLContext]() } @@ -77,8 +79,53 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ /** * Runs this query returning the result as an array. */ - def executeCollect(): Array[Row] = + def executeCollect(): Array[Row] = { execute().map(ScalaReflection.convertRowToScala(_, schema)).collect() + } + + /** + * Runs this query returning the first `n` rows as an array. + * + * This is modeled after RDD.take but never runs any job locally on the driver. + */ + def executeTake(n: Int): Array[Row] = { + if (n == 0) { + return new Array[Row](0) + } + + val childRDD = execute().map(_.copy()) + + val buf = new ArrayBuffer[Row] + val totalParts = childRDD.partitions.length + var partsScanned = 0 + while (buf.size < n && partsScanned < totalParts) { + // The number of partitions to try in this iteration. It is ok for this number to be + // greater than totalParts because we actually cap it at totalParts in runJob. + var numPartsToTry = 1 + if (partsScanned > 0) { + // If we didn't find any rows after the first iteration, just try all partitions next. + // Otherwise, interpolate the number of partitions we need to try, but overestimate it + // by 50%. + if (buf.size == 0) { + numPartsToTry = totalParts - 1 + } else { + numPartsToTry = (1.5 * n * partsScanned / buf.size).toInt + } + } + numPartsToTry = math.max(0, numPartsToTry) // guard against negative num of partitions + + val left = n - buf.size + val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts) + val sc = sqlContext.sparkContext + val res = + sc.runJob(childRDD, (it: Iterator[Row]) => it.take(left).toArray, p, allowLocal = false) + + res.foreach(buf ++= _.take(n - buf.size)) + partsScanned += numPartsToTry + } + + buf.toArray.map(ScalaReflection.convertRowToScala(_, this.schema)) + } protected def newProjection( expressions: Seq[Expression], inputSchema: Seq[Attribute]): Projection = { 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 ff0609d4b3b72..0c77d399b2eb8 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 @@ -21,7 +21,7 @@ import org.apache.spark.sql.{SQLContext, Strategy, execution} import org.apache.spark.sql.catalyst.expressions._ 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.catalyst.plans.logical.{Command, LogicalPlan} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} import org.apache.spark.sql.parquet._ @@ -284,13 +284,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Aggregate(partial = false, group, agg, planLater(child)) :: Nil case logical.Sample(fraction, withReplacement, seed, child) => execution.Sample(fraction, withReplacement, seed, planLater(child)) :: Nil - case SparkLogicalPlan(alreadyPlanned) => alreadyPlanned :: Nil case logical.LocalRelation(output, data) => - val nPartitions = if (data.isEmpty) 1 else numPartitions - PhysicalRDD( - output, - RDDConversions.productToRowRdd(sparkContext.parallelize(data, nPartitions), - StructType.fromAttributes(output))) :: Nil + LocalTableScan(output, data) :: Nil case logical.Limit(IntegerLiteral(limit), child) => execution.Limit(limit, planLater(child)) :: Nil case Unions(unionChildren) => 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 16ca4be5587c4..66aed5d5113d1 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 @@ -103,49 +103,7 @@ case class Limit(limit: Int, child: SparkPlan) override def output = child.output override def outputPartitioning = SinglePartition - /** - * A custom implementation modeled after the take function on RDDs but which never runs any job - * locally. This is to avoid shipping an entire partition of data in order to retrieve only a few - * rows. - */ - override def executeCollect(): Array[Row] = { - if (limit == 0) { - return new Array[Row](0) - } - - val childRDD = child.execute().map(_.copy()) - - val buf = new ArrayBuffer[Row] - val totalParts = childRDD.partitions.length - var partsScanned = 0 - while (buf.size < limit && partsScanned < totalParts) { - // The number of partitions to try in this iteration. It is ok for this number to be - // greater than totalParts because we actually cap it at totalParts in runJob. - var numPartsToTry = 1 - if (partsScanned > 0) { - // If we didn't find any rows after the first iteration, just try all partitions next. - // Otherwise, interpolate the number of partitions we need to try, but overestimate it - // by 50%. - if (buf.size == 0) { - numPartsToTry = totalParts - 1 - } else { - numPartsToTry = (1.5 * limit * partsScanned / buf.size).toInt - } - } - numPartsToTry = math.max(0, numPartsToTry) // guard against negative num of partitions - - val left = limit - buf.size - val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts) - val sc = sqlContext.sparkContext - val res = - sc.runJob(childRDD, (it: Iterator[Row]) => it.take(left).toArray, p, allowLocal = false) - - res.foreach(buf ++= _.take(limit - buf.size)) - partsScanned += numPartsToTry - } - - buf.toArray.map(ScalaReflection.convertRowToScala(_, this.schema)) - } + override def executeCollect(): Array[Row] = child.executeTake(limit) override def execute() = { val rdd: RDD[_ <: Product2[Boolean, Row]] = if (sortBasedShuffleOn) { 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 e1c9a2be7d20d..1bc53968c4ca3 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 @@ -58,6 +58,8 @@ case class ExecutedCommand(cmd: RunnableCommand) extends SparkPlan { override def executeCollect(): Array[Row] = sideEffectResult.toArray + override def executeTake(limit: Int): Array[Row] = sideEffectResult.take(limit).toArray + override def execute(): RDD[Row] = sqlContext.sparkContext.parallelize(sideEffectResult, 1) } From c23ac03c8c27e840498a192b088e00b27076765f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 4 Feb 2015 22:39:44 -0800 Subject: [PATCH 083/578] SPARK-5607: Update to Kryo 2.24.0 to avoid including objenesis 1.2. Our existing Kryo version actually embeds objenesis 1.2 classes in its jar, causing dependency conflicts during tests. This updates us to Kryo 2.24.0 (which was changed to not embed objenesis) to avoid this behavior. See the JIRA for more detail. Author: Patrick Wendell Closes #4383 from pwendell/SPARK-5607 and squashes the following commits: c3b8d27 [Patrick Wendell] SPARK-5607: Update to Kryo 2.24.0 to avoid including objenesis 1.2. --- pom.xml | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 55d82ad298def..d324b5f0ec93a 100644 --- a/pom.xml +++ b/pom.xml @@ -136,6 +136,7 @@ 1.2.3 8.1.14.v20131031 0.5.0 + 2.24.0 2.4.0 2.0.8 3.1.0 @@ -340,7 +341,13 @@
- + + + com.esotericsoftware.kryo + kryo + ${kryo.version} + + From 975bcef467b35586e5224171071355409f451d2d Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 4 Feb 2015 22:46:48 -0800 Subject: [PATCH 084/578] [SPARK-5596] [mllib] ML model import/export for GLMs, NaiveBayes This is a PR for Parquet-based model import/export. Please see the design doc on [the JIRA](https://issues.apache.org/jira/browse/SPARK-4587). Note: This includes only a subset of regression and classification models: * NaiveBayes, SVM, LogisticRegression * LinearRegression, RidgeRegression, Lasso Follow-up PRs will cover other models. Sketch of current contents: * New traits: Saveable, Loader * Implementations for some algorithms * Also: Added LogisticRegressionModel.getThreshold method (so that unit test could check the threshold) CC: mengxr selvinsource Author: Joseph K. Bradley Closes #4233 from jkbradley/ml-import-export and squashes the following commits: 87c4eb8 [Joseph K. Bradley] small cleanups 12d9059 [Joseph K. Bradley] Many cleanups after code review. Major changes: Storing numFeatures, numClasses in model metadata. Improvements to unit tests b4ee064 [Joseph K. Bradley] Reorganized save/load for regression and classification. Renamed concepts to Saveable, Loader a34aef5 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into ml-import-export ee99228 [Joseph K. Bradley] scala style fix 79675d5 [Joseph K. Bradley] cleanups in LogisticRegression after rebasing after multinomial PR d1e5882 [Joseph K. Bradley] organized imports 2935963 [Joseph K. Bradley] Added save/load and tests for most classification and regression models c495dba [Joseph K. Bradley] made version for model import/export local to each model 1496852 [Joseph K. Bradley] Added save/load for NaiveBayes 8d46386 [Joseph K. Bradley] Added save/load to NaiveBayes 1577d70 [Joseph K. Bradley] fixed issues after rebasing on master (DataFrame patch) 64914a3 [Joseph K. Bradley] added getThreshold to SVMModel b1fc5ec [Joseph K. Bradley] small cleanups 418ba1b [Joseph K. Bradley] Added save, load to mllib.classification.LogisticRegressionModel, plus test suite --- .../classification/ClassificationModel.scala | 20 +++ .../classification/LogisticRegression.scala | 67 ++++++++- .../mllib/classification/NaiveBayes.scala | 87 ++++++++++- .../spark/mllib/classification/SVM.scala | 51 ++++++- .../impl/GLMClassificationModel.scala | 95 ++++++++++++ .../apache/spark/mllib/regression/Lasso.scala | 33 ++++- .../mllib/regression/LinearRegression.scala | 35 ++++- .../mllib/regression/RegressionModel.scala | 22 ++- .../mllib/regression/RidgeRegression.scala | 38 ++++- .../regression/impl/GLMRegressionModel.scala | 86 +++++++++++ .../mllib/tree/model/DecisionTreeModel.scala | 1 - .../spark/mllib/util/modelSaveLoad.scala | 139 ++++++++++++++++++ .../LogisticRegressionSuite.scala | 70 ++++++++- .../classification/NaiveBayesSuite.scala | 40 ++++- .../spark/mllib/classification/SVMSuite.scala | 36 +++++ .../spark/mllib/regression/LassoSuite.scala | 24 +++ .../regression/LinearRegressionSuite.scala | 24 +++ .../regression/RidgeRegressionSuite.scala | 24 +++ 18 files changed, 863 insertions(+), 29 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala 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 b7a1d90d24d72..348c1e8760a66 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 @@ -20,7 +20,9 @@ package org.apache.spark.mllib.classification import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.util.Loader import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Row} /** * :: Experimental :: @@ -53,3 +55,21 @@ trait ClassificationModel extends Serializable { def predict(testData: JavaRDD[Vector]): JavaRDD[java.lang.Double] = predict(testData.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Double]] } + +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)}") + } + } + +} 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 a469315a1b5c3..5c9feb6fb2699 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -17,14 +17,17 @@ package org.apache.spark.mllib.classification +import org.apache.spark.SparkContext import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.classification.impl.GLMClassificationModel import org.apache.spark.mllib.linalg.BLAS.dot import org.apache.spark.mllib.linalg.{DenseVector, Vector} import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.{DataValidators, MLUtils} +import org.apache.spark.mllib.util.{DataValidators, Saveable, Loader} import org.apache.spark.rdd.RDD + /** * Classification model trained using Multinomial/Binary Logistic Regression. * @@ -42,7 +45,22 @@ class LogisticRegressionModel ( override val intercept: Double, val numFeatures: Int, val numClasses: Int) - extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable { + extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable + with Saveable { + + if (numClasses == 2) { + require(weights.size == numFeatures, + s"LogisticRegressionModel with numClasses = 2 was given non-matching values:" + + s" numFeatures = $numFeatures, but weights.size = ${weights.size}") + } else { + val weightsSizeWithoutIntercept = (numClasses - 1) * numFeatures + val weightsSizeWithIntercept = (numClasses - 1) * (numFeatures + 1) + require(weights.size == weightsSizeWithoutIntercept || weights.size == weightsSizeWithIntercept, + s"LogisticRegressionModel.load with numClasses = $numClasses and numFeatures = $numFeatures" + + s" expected weights of length $weightsSizeWithoutIntercept (without intercept)" + + s" or $weightsSizeWithIntercept (with intercept)," + + s" but was given weights of length ${weights.size}") + } def this(weights: Vector, intercept: Double) = this(weights, intercept, weights.size, 2) @@ -60,6 +78,13 @@ class LogisticRegressionModel ( this } + /** + * :: Experimental :: + * Returns the threshold (if any) used for converting raw prediction scores into 0/1 predictions. + */ + @Experimental + def getThreshold: Option[Double] = threshold + /** * :: Experimental :: * Clears the threshold so that `predict` will output raw prediction scores. @@ -70,7 +95,9 @@ class LogisticRegressionModel ( this } - override protected def predictPoint(dataMatrix: Vector, weightMatrix: Vector, + override protected def predictPoint( + dataMatrix: Vector, + weightMatrix: Vector, intercept: Double) = { require(dataMatrix.size == numFeatures) @@ -126,6 +153,40 @@ class LogisticRegressionModel ( bestClass.toDouble } } + + override def save(sc: SparkContext, path: String): Unit = { + GLMClassificationModel.SaveLoadV1_0.save(sc, path, this.getClass.getName, + numFeatures, numClasses, weights, intercept, threshold) + } + + override protected def formatVersion: String = "1.0" +} + +object LogisticRegressionModel extends Loader[LogisticRegressionModel] { + + override def load(sc: SparkContext, path: String): LogisticRegressionModel = { + val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) + // Hard-code class name string in case it changes in the future + 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 data = GLMClassificationModel.SaveLoadV1_0.loadData(sc, path, classNameV1_0) + // numFeatures, numClasses, weights are checked in model initialization + val model = + new LogisticRegressionModel(data.weights, data.intercept, numFeatures, numClasses) + data.threshold match { + case Some(t) => model.setThreshold(t) + case None => model.clearThreshold() + } + model + case _ => throw new Exception( + s"LogisticRegressionModel.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/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index a967df857bed3..4bafd495f90b1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -19,11 +19,13 @@ package org.apache.spark.mllib.classification import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum} -import org.apache.spark.{SparkException, Logging} -import org.apache.spark.SparkContext._ +import org.apache.spark.{SparkContext, SparkException, Logging} 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. @@ -36,7 +38,7 @@ import org.apache.spark.rdd.RDD class NaiveBayesModel private[mllib] ( val labels: Array[Double], val pi: Array[Double], - val theta: Array[Array[Double]]) extends ClassificationModel with Serializable { + val theta: Array[Array[Double]]) extends ClassificationModel with Serializable with Saveable { private val brzPi = new BDV[Double](pi) private val brzTheta = new BDM[Double](theta.length, theta(0).length) @@ -65,6 +67,85 @@ class NaiveBayesModel private[mllib] ( override def predict(testData: Vector): Double = { labels(brzArgmax(brzPi + brzTheta * testData.toBreeze)) } + + override def save(sc: SparkContext, path: String): Unit = { + val data = NaiveBayesModel.SaveLoadV1_0.Data(labels, pi, theta) + NaiveBayesModel.SaveLoadV1_0.save(sc, path, data) + } + + override protected def formatVersion: String = "1.0" +} + +object NaiveBayesModel extends Loader[NaiveBayesModel] { + + import Loader._ + + private 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.classification.NaiveBayesModel" + + /** Model data for model import/export */ + 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) + import sqlContext._ + + // 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)) + + // Create Parquet data. + val dataRDD: DataFrame = sc.parallelize(Seq(data), 1) + 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").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 + new NaiveBayesModel(labels, pi, theta) + } + } + + override def load(sc: SparkContext, path: String): NaiveBayesModel = { + val (loadedClassName, version, metadata) = loadMetadata(sc, path) + 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 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 + 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)") + } + } } /** 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 dd514ff8a37f2..24d31e62ba500 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -17,13 +17,16 @@ package org.apache.spark.mllib.classification +import org.apache.spark.SparkContext import org.apache.spark.annotation.Experimental +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 +import org.apache.spark.mllib.util.{DataValidators, Saveable, Loader} import org.apache.spark.rdd.RDD + /** * Model for Support Vector Machines (SVMs). * @@ -33,7 +36,8 @@ import org.apache.spark.rdd.RDD class SVMModel ( override val weights: Vector, override val intercept: Double) - extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable { + extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable + with Saveable { private var threshold: Option[Double] = Some(0.0) @@ -49,6 +53,13 @@ class SVMModel ( this } + /** + * :: Experimental :: + * Returns the threshold (if any) used for converting raw prediction scores into 0/1 predictions. + */ + @Experimental + def getThreshold: Option[Double] = threshold + /** * :: Experimental :: * Clears the threshold so that `predict` will output raw prediction scores. @@ -69,6 +80,42 @@ class SVMModel ( case None => margin } } + + override def save(sc: SparkContext, path: String): Unit = { + GLMClassificationModel.SaveLoadV1_0.save(sc, path, this.getClass.getName, + numFeatures = weights.size, numClasses = 2, weights, intercept, threshold) + } + + override protected def formatVersion: String = "1.0" +} + +object SVMModel extends Loader[SVMModel] { + + override def load(sc: SparkContext, path: String): SVMModel = { + val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) + // Hard-code class name string in case it changes in the future + 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 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" + + s" was given non-matching weights vector of size ${model.weights.size}") + assert(numClasses == 2, + s"SVMModel.load was given numClasses=$numClasses but only supports 2 classes") + data.threshold match { + case Some(t) => model.setThreshold(t) + case None => model.clearThreshold() + } + model + case _ => throw new Exception( + s"SVMModel.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/classification/impl/GLMClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala new file mode 100644 index 0000000000000..b60c0cdd0ab7b --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.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.mllib.classification.impl + +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} + +/** + * Helper class for import/export of GLM classification models. + */ +private[classification] object GLMClassificationModel { + + object SaveLoadV1_0 { + + def thisFormatVersion = "1.0" + + /** Model data for import/export */ + case class Data(weights: Vector, intercept: Double, threshold: Option[Double]) + + /** + * Helper method for saving GLM classification model metadata and data. + * @param modelClass String name for model class, to be saved with metadata + * @param numClasses Number of classes label can take, to be saved with metadata + */ + def save( + sc: SparkContext, + path: String, + modelClass: String, + numFeatures: Int, + numClasses: Int, + weights: Vector, + intercept: Double, + threshold: Option[Double]): Unit = { + val sqlContext = new SQLContext(sc) + import sqlContext._ + + // Create JSON metadata. + val metadataRDD = + sc.parallelize(Seq((modelClass, thisFormatVersion, numFeatures, numClasses)), 1) + .toDataFrame("class", "version", "numFeatures", "numClasses") + metadataRDD.toJSON.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)) + } + + /** + * Helper method for loading GLM classification model data. + * + * NOTE: Callers of this method should check numClasses, numFeatures on their own. + * + * @param modelClass String name for model class (used for error messages) + */ + def loadData(sc: SparkContext, path: String, modelClass: String): Data = { + val datapath = Loader.dataPath(path) + val sqlContext = new SQLContext(sc) + val dataRDD = sqlContext.parquetFile(datapath) + val dataArray = dataRDD.select("weights", "intercept", "threshold").take(1) + assert(dataArray.size == 1, s"Unable to load $modelClass data from: $datapath") + val data = dataArray(0) + assert(data.size == 3, s"Unable to load $modelClass data from: $datapath") + val (weights, intercept) = data match { + case Row(weights: Vector, intercept: Double, _) => + (weights, intercept) + } + val threshold = if (data.isNullAt(2)) { + None + } else { + Some(data.getDouble(2)) + } + Data(weights, intercept, threshold) + } + } + +} 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 8ecd5c6ad93c0..1159e59fff5f6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -17,9 +17,11 @@ package org.apache.spark.mllib.regression -import org.apache.spark.annotation.Experimental +import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.regression.impl.GLMRegressionModel +import org.apache.spark.mllib.util.{Saveable, Loader} import org.apache.spark.rdd.RDD /** @@ -32,7 +34,7 @@ class LassoModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) - with RegressionModel with Serializable { + with RegressionModel with Serializable with Saveable { override protected def predictPoint( dataMatrix: Vector, @@ -40,12 +42,37 @@ class LassoModel ( intercept: Double): Double = { weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept } + + override def save(sc: SparkContext, path: String): Unit = { + GLMRegressionModel.SaveLoadV1_0.save(sc, path, this.getClass.getName, weights, intercept) + } + + override protected def formatVersion: String = "1.0" +} + +object LassoModel extends Loader[LassoModel] { + + override def load(sc: SparkContext, path: String): LassoModel = { + val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) + // Hard-code class name string in case it changes in the future + 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 data = GLMRegressionModel.SaveLoadV1_0.loadData(sc, path, classNameV1_0, numFeatures) + new LassoModel(data.weights, data.intercept) + case _ => throw new Exception( + s"LassoModel.load did not recognize model with (className, format version):" + + s"($loadedClassName, $version). Supported:\n" + + s" ($classNameV1_0, 1.0)") + } + } } /** * Train a regression model with L1-regularization using Stochastic Gradient Descent. * This solves the l1-regularized least squares regression formulation - * f(weights) = 1/2n ||A weights-y||^2 + regParam ||weights||_1 + * f(weights) = 1/2n ||A weights-y||^2^ + regParam ||weights||_1 * Here the data matrix has n rows, and the input RDD holds the set of rows of A, each with * its corresponding right hand side label y. * See also the documentation for the precise formulation. 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 81b6598377ff5..0136dcfdceaef 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -17,9 +17,12 @@ package org.apache.spark.mllib.regression -import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.regression.impl.GLMRegressionModel +import org.apache.spark.mllib.util.{Saveable, Loader} +import org.apache.spark.rdd.RDD /** * Regression model trained using LinearRegression. @@ -30,7 +33,8 @@ import org.apache.spark.mllib.optimization._ class LinearRegressionModel ( override val weights: Vector, override val intercept: Double) - extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { + extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable + with Saveable { override protected def predictPoint( dataMatrix: Vector, @@ -38,12 +42,37 @@ class LinearRegressionModel ( intercept: Double): Double = { weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept } + + override def save(sc: SparkContext, path: String): Unit = { + GLMRegressionModel.SaveLoadV1_0.save(sc, path, this.getClass.getName, weights, intercept) + } + + override protected def formatVersion: String = "1.0" +} + +object LinearRegressionModel extends Loader[LinearRegressionModel] { + + override def load(sc: SparkContext, path: String): LinearRegressionModel = { + val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) + // Hard-code class name string in case it changes in the future + 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 data = GLMRegressionModel.SaveLoadV1_0.loadData(sc, path, classNameV1_0, numFeatures) + new LinearRegressionModel(data.weights, data.intercept) + case _ => throw new Exception( + s"LinearRegressionModel.load did not recognize model with (className, format version):" + + s"($loadedClassName, $version). Supported:\n" + + s" ($classNameV1_0, 1.0)") + } + } } /** * Train a linear regression model with no regularization using Stochastic Gradient Descent. * This solves the least squares regression formulation - * f(weights) = 1/n ||A weights-y||^2 + * f(weights) = 1/n ||A weights-y||^2^ * (which is the mean squared error). * Here the data matrix has n rows, and the input RDD holds the set of rows of A, each with * its corresponding right hand side label y. 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 64b02f7a6e7a9..843e59bdfbdd2 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 @@ -19,8 +19,10 @@ package org.apache.spark.mllib.regression import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD -import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.util.Loader +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, Row} @Experimental trait RegressionModel extends Serializable { @@ -48,3 +50,21 @@ trait RegressionModel extends Serializable { def predict(testData: JavaRDD[Vector]): JavaRDD[java.lang.Double] = predict(testData.rdd).toJavaRDD().asInstanceOf[JavaRDD[java.lang.Double]] } + +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)}") + } + } + +} 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 076ba35051c9d..f2a5f1db1ece6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -17,10 +17,13 @@ package org.apache.spark.mllib.regression -import org.apache.spark.annotation.Experimental -import org.apache.spark.rdd.RDD -import org.apache.spark.mllib.optimization._ +import org.apache.spark.SparkContext import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.regression.impl.GLMRegressionModel +import org.apache.spark.mllib.util.{Loader, Saveable} +import org.apache.spark.rdd.RDD + /** * Regression model trained using RidgeRegression. @@ -32,7 +35,7 @@ class RidgeRegressionModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) - with RegressionModel with Serializable { + with RegressionModel with Serializable with Saveable { override protected def predictPoint( dataMatrix: Vector, @@ -40,12 +43,37 @@ class RidgeRegressionModel ( intercept: Double): Double = { weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept } + + override def save(sc: SparkContext, path: String): Unit = { + GLMRegressionModel.SaveLoadV1_0.save(sc, path, this.getClass.getName, weights, intercept) + } + + override protected def formatVersion: String = "1.0" +} + +object RidgeRegressionModel extends Loader[RidgeRegressionModel] { + + override def load(sc: SparkContext, path: String): RidgeRegressionModel = { + val (loadedClassName, version, metadata) = Loader.loadMetadata(sc, path) + // Hard-code class name string in case it changes in the future + 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 data = GLMRegressionModel.SaveLoadV1_0.loadData(sc, path, classNameV1_0, numFeatures) + new RidgeRegressionModel(data.weights, data.intercept) + case _ => throw new Exception( + s"RidgeRegressionModel.load did not recognize model with (className, format version):" + + s"($loadedClassName, $version). Supported:\n" + + s" ($classNameV1_0, 1.0)") + } + } } /** * Train a regression model with L2-regularization using Stochastic Gradient Descent. * This solves the l1-regularized least squares regression formulation - * f(weights) = 1/2n ||A weights-y||^2 + regParam/2 ||weights||^2 + * f(weights) = 1/2n ||A weights-y||^2^ + regParam/2 ||weights||^2^ * Here the data matrix has n rows, and the input RDD holds the set of rows of A, each with * its corresponding right hand side label y. * See also the documentation for the precise formulation. 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 new file mode 100644 index 0000000000000..00f25a8be9397 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.regression.impl + +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} + +/** + * Helper methods for import/export of GLM regression models. + */ +private[regression] object GLMRegressionModel { + + object SaveLoadV1_0 { + + def thisFormatVersion = "1.0" + + /** Model data for model import/export */ + case class Data(weights: Vector, intercept: Double) + + /** + * Helper method for saving GLM regression model metadata and data. + * @param modelClass String name for model class, to be saved with metadata + */ + def save( + sc: SparkContext, + path: String, + modelClass: String, + weights: Vector, + intercept: Double): Unit = { + val sqlContext = new SQLContext(sc) + import sqlContext._ + + // Create JSON metadata. + val metadataRDD = + sc.parallelize(Seq((modelClass, thisFormatVersion, weights.size)), 1) + .toDataFrame("class", "version", "numFeatures") + metadataRDD.toJSON.saveAsTextFile(Loader.metadataPath(path)) + + // Create Parquet data. + val data = Data(weights, intercept) + val dataRDD: DataFrame = sc.parallelize(Seq(data), 1) + // TODO: repartition with 1 partition after SPARK-5532 gets fixed + dataRDD.saveAsParquetFile(Loader.dataPath(path)) + } + + /** + * Helper method for loading GLM regression model data. + * @param modelClass String name for model class (used for error messages) + * @param numFeatures Number of features, to be checked against loaded data. + * The length of the weights vector should equal numFeatures. + */ + def loadData(sc: SparkContext, path: String, modelClass: String, numFeatures: Int): Data = { + val datapath = Loader.dataPath(path) + val sqlContext = new SQLContext(sc) + val dataRDD = sqlContext.parquetFile(datapath) + val dataArray = dataRDD.select("weights", "intercept").take(1) + assert(dataArray.size == 1, s"Unable to load $modelClass data from: $datapath") + val data = dataArray(0) + assert(data.size == 2, s"Unable to load $modelClass data from: $datapath") + data match { + case Row(weights: Vector, intercept: Double) => + assert(weights.size == numFeatures, s"Expected $numFeatures features, but" + + s" found ${weights.size} features when loading $modelClass weights from $datapath") + Data(weights, intercept) + } + } + } + +} 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 a5760963068c3..a25e625a4017a 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 @@ -53,7 +53,6 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable features.map(x => predict(x)) } - /** * Predict values for the given data set using the model trained. * 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 new file mode 100644 index 0000000000000..56b77a7d12e83 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/modelSaveLoad.scala @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.util + +import scala.reflect.runtime.universe.TypeTag + +import org.apache.hadoop.fs.Path + +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} + + +/** + * :: DeveloperApi :: + * + * Trait for models and transformers which may be saved as files. + * This should be inherited by the class which implements model instances. + */ +@DeveloperApi +trait Saveable { + + /** + * 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 [[Loader.load]]. + * + * @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. + */ + def save(sc: SparkContext, path: String): Unit + + /** Current version of model save/load format. */ + protected def formatVersion: String + +} + +/** + * :: DeveloperApi :: + * + * Trait for classes which can load models and transformers from files. + * This should be inherited by an object paired with the model class. + */ +@DeveloperApi +trait Loader[M <: Saveable] { + + /** + * Load a model from the given path. + * + * The model should have been saved by [[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 + */ + def load(sc: SparkContext, path: String): M + +} + +/** + * Helper methods for loading models from files. + */ +private[mllib] object Loader { + + /** Returns URI for path/data using the Hadoop filesystem */ + def dataPath(path: String): String = new Path(path, "data").toUri.toString + + /** Returns URI for path/metadata using the Hadoop filesystem */ + def metadataPath(path: String): String = new Path(path, "metadata").toUri.toString + + /** + * Check the schema of loaded model data. + * + * This checks every field in the expected schema to make sure that a field with the same + * name and DataType appears in the loaded schema. Note that this does NOT check metadata + * or containsNull. + * + * @param loadedSchema Schema for model data loaded from file. + * @tparam Data Expected data type from which an expected schema can be derived. + */ + def checkSchema[Data: TypeTag](loadedSchema: StructType): Unit = { + // Check schema explicitly since erasure makes it hard to use match-case for checking. + val expectedFields: Array[StructField] = + ScalaReflection.schemaFor[Data].dataType.asInstanceOf[StructType].fields + val loadedFields: Map[String, DataType] = + loadedSchema.map(field => field.name -> field.dataType).toMap + expectedFields.foreach { field => + 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, + s"Unable to parse model data. Expected field $field but found field" + + s" with different type: ${loadedFields(field.name)}") + } + } + + /** + * 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)}") + } + (clazz, version, metadata) + } + +} 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 3fb45938f75db..d2b40f2cae020 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 @@ -17,9 +17,9 @@ package org.apache.spark.mllib.classification -import scala.util.control.Breaks._ -import scala.util.Random import scala.collection.JavaConversions._ +import scala.util.Random +import scala.util.control.Breaks._ import org.scalatest.FunSuite import org.scalatest.Matchers @@ -28,6 +28,8 @@ import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.{LocalClusterSparkContext, MLlibTestSparkContext} import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.util.Utils + object LogisticRegressionSuite { @@ -147,8 +149,25 @@ object LogisticRegressionSuite { val testData = (0 until nPoints).map(i => LabeledPoint(y(i), x(i))) testData } + + /** Binary labels, 3 features */ + private val binaryModel = new LogisticRegressionModel( + weights = Vectors.dense(0.1, 0.2, 0.3), intercept = 0.5, numFeatures = 3, numClasses = 2) + + /** 3 classes, 2 features */ + private val multiclassModel = new LogisticRegressionModel( + weights = Vectors.dense(0.1, 0.2, 0.3, 0.4), intercept = 1.0, numFeatures = 2, numClasses = 3) + + private def checkModelsEqual(a: LogisticRegressionModel, b: LogisticRegressionModel): Unit = { + assert(a.weights == b.weights) + assert(a.intercept == b.intercept) + assert(a.numClasses == b.numClasses) + assert(a.numFeatures == b.numFeatures) + assert(a.getThreshold == b.getThreshold) + } } + class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext with Matchers { def validatePrediction( predictions: Seq[Double], @@ -462,6 +481,53 @@ class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext with M } + test("model save/load: binary classification") { + // NOTE: This will need to be generalized once there are multiple model format versions. + val model = LogisticRegressionSuite.binaryModel + + model.clearThreshold() + assert(model.getThreshold.isEmpty) + + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + // Save model, load it back, and compare. + try { + model.save(sc, path) + val sameModel = LogisticRegressionModel.load(sc, path) + LogisticRegressionSuite.checkModelsEqual(model, sameModel) + } finally { + Utils.deleteRecursively(tempDir) + } + + // Save model with threshold. + try { + model.setThreshold(0.7) + model.save(sc, path) + val sameModel = LogisticRegressionModel.load(sc, path) + LogisticRegressionSuite.checkModelsEqual(model, sameModel) + } finally { + Utils.deleteRecursively(tempDir) + } + } + + test("model save/load: multiclass classification") { + // NOTE: This will need to be generalized once there are multiple model format versions. + val model = LogisticRegressionSuite.multiclassModel + + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + // Save model, load it back, and compare. + try { + model.save(sc, path) + val sameModel = LogisticRegressionModel.load(sc, path) + LogisticRegressionSuite.checkModelsEqual(model, sameModel) + } finally { + Utils.deleteRecursively(tempDir) + } + } + } class LogisticRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { 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 e68fe89d6ccea..64dcc0fb9f82c 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 @@ -25,6 +25,8 @@ import org.apache.spark.SparkException import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.{LocalClusterSparkContext, MLlibTestSparkContext} +import org.apache.spark.util.Utils + object NaiveBayesSuite { @@ -58,6 +60,18 @@ object NaiveBayesSuite { LabeledPoint(y, Vectors.dense(xi)) } } + + private val smallPi = Array(0.5, 0.3, 0.2).map(math.log) + + 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))) } class NaiveBayesSuite extends FunSuite with MLlibTestSparkContext { @@ -74,12 +88,8 @@ class NaiveBayesSuite extends FunSuite with MLlibTestSparkContext { test("Naive Bayes") { val nPoints = 10000 - val pi = Array(0.5, 0.3, 0.2).map(math.log) - val theta = 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)) + val pi = NaiveBayesSuite.smallPi + val theta = NaiveBayesSuite.smallTheta val testData = NaiveBayesSuite.generateNaiveBayesInput(pi, theta, nPoints, 42) val testRDD = sc.parallelize(testData, 2) @@ -123,6 +133,24 @@ class NaiveBayesSuite extends FunSuite with MLlibTestSparkContext { NaiveBayes.train(sc.makeRDD(nan, 2)) } } + + test("model save/load") { + val model = NaiveBayesSuite.binaryModel + + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + // 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) + } finally { + Utils.deleteRecursively(tempDir) + } + } } class NaiveBayesClusterSuite extends FunSuite with LocalClusterSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala index a2de7fbd41383..6de098b383ba3 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.SparkException import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.{LocalClusterSparkContext, MLlibTestSparkContext} +import org.apache.spark.util.Utils object SVMSuite { @@ -56,6 +57,9 @@ object SVMSuite { y.zip(x).map(p => LabeledPoint(p._1, Vectors.dense(p._2))) } + /** Binary labels, 3 features */ + private val binaryModel = new SVMModel(weights = Vectors.dense(0.1, 0.2, 0.3), intercept = 0.5) + } class SVMSuite extends FunSuite with MLlibTestSparkContext { @@ -191,6 +195,38 @@ class SVMSuite extends FunSuite with MLlibTestSparkContext { // Turning off data validation should not throw an exception new SVMWithSGD().setValidateData(false).run(testRDDInvalid) } + + test("model save/load") { + // NOTE: This will need to be generalized once there are multiple model format versions. + val model = SVMSuite.binaryModel + + model.clearThreshold() + assert(model.getThreshold.isEmpty) + + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + // Save model, load it back, and compare. + try { + model.save(sc, path) + val sameModel = SVMModel.load(sc, path) + assert(model.weights == sameModel.weights) + assert(model.intercept == sameModel.intercept) + assert(sameModel.getThreshold.isEmpty) + } finally { + Utils.deleteRecursively(tempDir) + } + + // Save model with threshold. + try { + model.setThreshold(0.7) + model.save(sc, path) + val sameModel2 = SVMModel.load(sc, path) + assert(model.getThreshold.get == sameModel2.getThreshold.get) + } finally { + Utils.deleteRecursively(tempDir) + } + } } class SVMClusterSuite extends FunSuite with LocalClusterSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala index 2668dcc14a842..c9f5dc069ef2e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala @@ -24,6 +24,13 @@ import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, MLlibTestSparkContext} +import org.apache.spark.util.Utils + +private object LassoSuite { + + /** 3 features */ + val model = new LassoModel(weights = Vectors.dense(0.1, 0.2, 0.3), intercept = 0.5) +} class LassoSuite extends FunSuite with MLlibTestSparkContext { @@ -115,6 +122,23 @@ class LassoSuite extends FunSuite with MLlibTestSparkContext { // Test prediction on Array. validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } + + test("model save/load") { + val model = LassoSuite.model + + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + // Save model, load it back, and compare. + try { + model.save(sc, path) + val sameModel = LassoModel.load(sc, path) + assert(model.weights == sameModel.weights) + assert(model.intercept == sameModel.intercept) + } finally { + Utils.deleteRecursively(tempDir) + } + } } class LassoClusterSuite extends FunSuite with LocalClusterSparkContext { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala index 864622a9296a6..3781931c2f819 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala @@ -24,6 +24,13 @@ import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, MLlibTestSparkContext} +import org.apache.spark.util.Utils + +private object LinearRegressionSuite { + + /** 3 features */ + val model = new LinearRegressionModel(weights = Vectors.dense(0.1, 0.2, 0.3), intercept = 0.5) +} class LinearRegressionSuite extends FunSuite with MLlibTestSparkContext { @@ -124,6 +131,23 @@ class LinearRegressionSuite extends FunSuite with MLlibTestSparkContext { validatePrediction( sparseValidationData.map(row => model.predict(row.features)), sparseValidationData) } + + test("model save/load") { + val model = LinearRegressionSuite.model + + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + // Save model, load it back, and compare. + try { + model.save(sc, path) + val sameModel = LinearRegressionModel.load(sc, path) + assert(model.weights == sameModel.weights) + assert(model.intercept == sameModel.intercept) + } finally { + Utils.deleteRecursively(tempDir) + } + } } class LinearRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { 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 18d3bf5ea4eca..43d61151e2471 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 @@ -25,6 +25,13 @@ import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, MLlibTestSparkContext} +import org.apache.spark.util.Utils + +private object RidgeRegressionSuite { + + /** 3 features */ + val model = new RidgeRegressionModel(weights = Vectors.dense(0.1, 0.2, 0.3), intercept = 0.5) +} class RidgeRegressionSuite extends FunSuite with MLlibTestSparkContext { @@ -75,6 +82,23 @@ class RidgeRegressionSuite extends FunSuite with MLlibTestSparkContext { assert(ridgeErr < linearErr, "ridgeError (" + ridgeErr + ") was not less than linearError(" + linearErr + ")") } + + test("model save/load") { + val model = RidgeRegressionSuite.model + + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + + // Save model, load it back, and compare. + try { + model.save(sc, path) + val sameModel = RidgeRegressionModel.load(sc, path) + assert(model.weights == sameModel.weights) + assert(model.intercept == sameModel.intercept) + } finally { + Utils.deleteRecursively(tempDir) + } + } } class RidgeRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { From db34690466d67f9c8ac6a145fddb5f7ea30a8d8d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 4 Feb 2015 23:03:47 -0800 Subject: [PATCH 085/578] [SPARK-5599] Check MLlib public APIs for 1.3 There are no break changes (against 1.2) in this PR. I hide the PythonMLLibAPI, which is only called by Py4J, and renamed `SparseMatrix.diag` to `SparseMatrix.spdiag`. All other changes are documentation and annotations. The `Experimental` tag is removed from `ALS.setAlpha` and `Rating`. One issue not addressed in this PR is the `setCheckpointDir` in `LDA` (https://issues.apache.org/jira/browse/SPARK-5604). CC: srowen jkbradley Author: Xiangrui Meng Closes #4377 from mengxr/SPARK-5599 and squashes the following commits: 17975dc [Xiangrui Meng] fix tests 4487f20 [Xiangrui Meng] remove experimental tag from each stat method because Statistics is experimental already 3cd969a [Xiangrui Meng] remove freeman (sorry~) from StreamLA public doc 55900f5 [Xiangrui Meng] make IR experimental and update its doc 9b8eed3 [Xiangrui Meng] graduate Rating and setAlpha in ALS b854d28 [Xiangrui Meng] correct iid doc in RandomRDDs 27f5bdd [Xiangrui Meng] update linalg docs and some new method signatures 371721b [Xiangrui Meng] mark fpg as experimental and update its doc 8aca7ee [Xiangrui Meng] change SLR to experimental and update the doc ebbb2e9 [Xiangrui Meng] mark PIC experimental and update the doc 7830d3b [Xiangrui Meng] mark GMM experimental a378496 [Xiangrui Meng] use the correct subscript syntax in PIC c65c424 [Xiangrui Meng] update LDAModel doc a213b0c [Xiangrui Meng] update GMM constructor 3993054 [Xiangrui Meng] hide algorithm in SLR ad6b9ce [Xiangrui Meng] Revert "make ClassificatinModel.predict(JavaRDD) return JavaDoubleRDD" 0054684 [Xiangrui Meng] add doc to LRModel's constructor a89763b [Xiangrui Meng] make ClassificatinModel.predict(JavaRDD) return JavaDoubleRDD 7c0946c [Xiangrui Meng] hide PythonMLLibAPI --- .../mllib/api/python/PythonMLLibAPI.scala | 6 +- .../classification/LogisticRegression.scala | 3 + .../StreamingLogisticRegressionWithSGD.scala | 5 +- .../mllib/clustering/GaussianMixture.scala | 13 ++++- .../clustering/GaussianMixtureModel.scala | 6 +- .../spark/mllib/clustering/LDAModel.scala | 2 +- .../clustering/PowerIterationClustering.scala | 26 ++++++--- .../mllib/clustering/StreamingKMeans.scala | 16 ++++-- .../org/apache/spark/mllib/fpm/FPGrowth.scala | 27 ++++++--- .../apache/spark/mllib/linalg/Matrices.scala | 48 +++++++++------- .../apache/spark/mllib/linalg/Vectors.scala | 4 +- .../linalg/distributed/BlockMatrix.scala | 12 +++- .../spark/mllib/random/RandomRDDs.scala | 57 ++++++++++--------- .../spark/mllib/recommendation/ALS.scala | 4 -- .../mllib/regression/IsotonicRegression.scala | 13 ++++- .../regression/StreamingLinearAlgorithm.scala | 4 +- .../apache/spark/mllib/stat/Statistics.scala | 21 +------ .../spark/mllib/linalg/JavaMatricesSuite.java | 4 +- .../spark/mllib/linalg/MatricesSuite.scala | 8 +-- 19 files changed, 160 insertions(+), 119 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 3f29b82ce85ac..cbd87ea8aeb37 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 @@ -54,11 +54,9 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils /** - * :: DeveloperApi :: - * The Java stubs necessary for the Python mllib bindings. + * The Java stubs necessary for the Python mllib bindings. It is called by Py4J on the Python side. */ -@DeveloperApi -class PythonMLLibAPI extends Serializable { +private[python] class PythonMLLibAPI extends Serializable { /** 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 5c9feb6fb2699..a668e7a7a359f 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,9 @@ class LogisticRegressionModel ( s" but was given weights of length ${weights.size}") } + /** + * Constructs a [[LogisticRegressionModel]] with weights and intercept for binary classification. + */ def this(weights: Vector, intercept: Double) = this(weights, intercept, weights.size, 2) private var threshold: Option[Double] = Some(0.5) 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 6a3893d0e41d2..b89f38cf5aba4 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 @@ -35,12 +35,13 @@ import org.apache.spark.mllib.regression.StreamingLinearAlgorithm * Use a builder pattern to construct a streaming logistic regression * analysis in an application, like: * + * {{{ * val model = new StreamingLogisticRegressionWithSGD() * .setStepSize(0.5) * .setNumIterations(10) * .setInitialWeights(Vectors.dense(...)) * .trainOn(DStream) - * + * }}} */ @Experimental class StreamingLogisticRegressionWithSGD private[mllib] ( @@ -59,7 +60,7 @@ class StreamingLogisticRegressionWithSGD private[mllib] ( */ def this() = this(0.1, 50, 1.0, 0.0) - val algorithm = new LogisticRegressionWithSGD( + protected val algorithm = new LogisticRegressionWithSGD( stepSize, numIterations, regParam, miniBatchFraction) /** Set the step size for gradient descent. Default: 0.1. */ 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 5c626fde4e657..0be3014de862e 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,15 +19,18 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable.IndexedSeq -import breeze.linalg.{DenseVector => BreezeVector, DenseMatrix => BreezeMatrix, diag, Transpose} +import breeze.linalg.{DenseMatrix => BreezeMatrix, DenseVector => BreezeVector, Transpose, diag} -import org.apache.spark.mllib.linalg.{Matrices, Vector, Vectors, DenseVector, DenseMatrix, BLAS} +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.{BLAS, DenseMatrix, DenseVector, Matrices, Vector, Vectors} import org.apache.spark.mllib.stat.distribution.MultivariateGaussian import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils /** + * :: Experimental :: + * * This class performs expectation maximization for multivariate Gaussian * Mixture Models (GMMs). A GMM represents a composite distribution of * independent Gaussian distributions with associated "mixing" weights @@ -44,13 +47,17 @@ import org.apache.spark.util.Utils * is considered to have occurred. * @param maxIterations The maximum number of iterations to perform */ +@Experimental class GaussianMixture private ( private var k: Int, private var convergenceTol: Double, private var maxIterations: Int, private var seed: Long) extends Serializable { - /** A default instance, 2 Gaussians, 100 iterations, 0.01 log-likelihood threshold */ + /** + * Constructs a default instance. The default parameters are {k: 2, convergenceTol: 0.01, + * maxIterations: 100, seed: random}. + */ def this() = this(2, 0.01, 100, Utils.random.nextLong()) // number of samples per cluster to use when initializing Gaussians diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index 1a2178ee7f711..af6f83c74bb40 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -19,12 +19,15 @@ package org.apache.spark.mllib.clustering import breeze.linalg.{DenseVector => BreezeVector} -import org.apache.spark.rdd.RDD +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.stat.distribution.MultivariateGaussian import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD /** + * :: Experimental :: + * * Multivariate Gaussian Mixture Model (GMM) consisting of k Gaussians, where points * are drawn from each Gaussian i=1..k with probability w(i); mu(i) and sigma(i) are * the respective mean and covariance for each Gaussian distribution i=1..k. @@ -35,6 +38,7 @@ import org.apache.spark.mllib.util.MLUtils * @param sigma Covariance maxtrix for each Gaussian in the mixture, where sigma(i) is the * covariance matrix for Gaussian i */ +@Experimental class GaussianMixtureModel( val weights: Array[Double], val gaussians: Array[MultivariateGaussian]) extends Serializable { 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 19e8aab6eabd7..b0e991d2f2344 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 @@ -335,7 +335,7 @@ class DistributedLDAModel private ( /** * For each document in the training set, return the distribution over topics for that document - * (i.e., "theta_doc"). + * ("theta_doc"). * * @return RDD of (document ID, topic distribution) pairs */ 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 9b5c155b0a805..3b1caf0c679ef 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 @@ -18,6 +18,7 @@ package org.apache.spark.mllib.clustering import org.apache.spark.{Logging, SparkException} +import org.apache.spark.annotation.Experimental import org.apache.spark.graphx._ import org.apache.spark.graphx.impl.GraphImpl import org.apache.spark.mllib.linalg.Vectors @@ -26,25 +27,33 @@ import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom /** + * :: Experimental :: + * * Model produced by [[PowerIterationClustering]]. * * @param k number of clusters * @param assignments an RDD of (vertexID, clusterID) pairs */ +@Experimental class PowerIterationClusteringModel( val k: Int, val assignments: RDD[(Long, Int)]) extends Serializable /** - * Power Iteration Clustering (PIC), a scalable graph clustering algorithm developed by Lin and - * Cohen (see http://www.icml2010.org/papers/387.pdf). From the abstract: PIC finds a very + * :: Experimental :: + * + * Power Iteration Clustering (PIC), a scalable graph clustering algorithm developed by + * [[http://www.icml2010.org/papers/387.pdf Lin and Cohen]]. From the abstract: PIC finds a very * low-dimensional embedding of a dataset using truncated power iteration on a normalized pair-wise * similarity matrix of the data. * * @param k Number of clusters. * @param maxIterations Maximum number of iterations of the PIC algorithm. * @param initMode Initialization mode. + * + * @see [[http://en.wikipedia.org/wiki/Spectral_clustering Spectral clustering (Wikipedia)]] */ +@Experimental class PowerIterationClustering private[clustering] ( private var k: Int, private var maxIterations: Int, @@ -88,11 +97,12 @@ class PowerIterationClustering private[clustering] ( /** * Run the PIC algorithm. * - * @param similarities an RDD of (i, j, s_ij_) tuples representing the affinity matrix, which is - * the matrix A in the PIC paper. The similarity s_ij_ must be nonnegative. - * This is a symmetric matrix and hence s_ij_ = s_ji_. For any (i, j) with - * nonzero similarity, there should be either (i, j, s_ij_) or (j, i, s_ji_) - * in the input. Tuples with i = j are ignored, because we assume s_ij_ = 0.0. + * @param similarities an RDD of (i, j, s,,ij,,) tuples representing the affinity matrix, which is + * the matrix A in the PIC paper. The similarity s,,ij,, must be nonnegative. + * This is a symmetric matrix and hence s,,ij,, = s,,ji,,. For any (i, j) with + * nonzero similarity, there should be either (i, j, s,,ij,,) or + * (j, i, s,,ji,,) in the input. Tuples with i = j are ignored, because we + * assume s,,ij,, = 0.0. * * @return a [[PowerIterationClusteringModel]] that contains the clustering result */ @@ -109,7 +119,7 @@ class PowerIterationClustering private[clustering] ( * Runs the PIC algorithm. * * @param w The normalized affinity matrix, which is the matrix W in the PIC paper with - * w_ij_ = a_ij_ / d_ii_ as its edge properties and the initial vector of the power + * w,,ij,, = a,,ij,, / d,,ii,, as its edge properties and the initial vector of the power * iteration as its vertex properties. */ private def pic(w: Graph[Double, Double]): PowerIterationClusteringModel = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala index 7752c1988fdd1..f483fd1c7d2cf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala @@ -21,7 +21,7 @@ import scala.reflect.ClassTag import org.apache.spark.Logging import org.apache.spark.SparkContext._ -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{Experimental, DeveloperApi} import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.rdd.RDD import org.apache.spark.streaming.dstream.DStream @@ -29,7 +29,8 @@ import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom /** - * :: DeveloperApi :: + * :: Experimental :: + * * StreamingKMeansModel extends MLlib's KMeansModel for streaming * algorithms, so it can keep track of a continuously updated weight * associated with each cluster, and also update the model by @@ -39,8 +40,10 @@ import org.apache.spark.util.random.XORShiftRandom * generalized to incorporate forgetfullness (i.e. decay). * The update rule (for each cluster) is: * + * {{{ * c_t+1 = [(c_t * n_t * a) + (x_t * m_t)] / [n_t + m_t] * n_t+t = n_t * a + m_t + * }}} * * Where c_t is the previously estimated centroid for that cluster, * n_t is the number of points assigned to it thus far, x_t is the centroid @@ -61,7 +64,7 @@ import org.apache.spark.util.random.XORShiftRandom * as batches or points. * */ -@DeveloperApi +@Experimental class StreamingKMeansModel( override val clusterCenters: Array[Vector], val clusterWeights: Array[Double]) extends KMeansModel(clusterCenters) with Logging { @@ -140,7 +143,8 @@ class StreamingKMeansModel( } /** - * :: DeveloperApi :: + * :: Experimental :: + * * StreamingKMeans provides methods for configuring a * streaming k-means analysis, training the model on streaming, * and using the model to make predictions on streaming data. @@ -149,13 +153,15 @@ class StreamingKMeansModel( * Use a builder pattern to construct a streaming k-means analysis * in an application, like: * + * {{{ * val model = new StreamingKMeans() * .setDecayFactor(0.5) * .setK(3) * .setRandomCenters(5, 100.0) * .trainOn(DStream) + * }}} */ -@DeveloperApi +@Experimental class StreamingKMeans( var k: Int, var decayFactor: Double, 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 1433ee9a0dd5a..3168d608c9556 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 @@ -25,16 +25,20 @@ import scala.collection.JavaConverters._ 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.JavaSparkContext.fakeClassTag import org.apache.spark.rdd.RDD 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 * @tparam Item item type */ +@Experimental class FPGrowthModel[Item: ClassTag]( val freqItemsets: RDD[(Array[Item], Long)]) extends Serializable { @@ -45,28 +49,35 @@ class FPGrowthModel[Item: ClassTag]( } /** - * This class implements Parallel FP-growth algorithm to do frequent pattern matching on input data. - * Parallel FPGrowth (PFP) partitions computation in such a way that each machine executes an - * independent group of mining tasks. More detail of this algorithm can be found at - * [[http://dx.doi.org/10.1145/1454008.1454027, PFP]], and the original FP-growth paper can be - * found at [[http://dx.doi.org/10.1145/335191.335372, FP-growth]] + * :: Experimental :: + * + * A parallel FP-growth algorithm to mine frequent itemsets. The algorithm is described in + * [[http://dx.doi.org/10.1145/1454008.1454027 Li et al., PFP: Parallel FP-Growth for Query + * Recommendation]]. PFP distributes computation in such a way that each worker executes an + * independent group of mining tasks. The FP-Growth algorithm is described in + * [[http://dx.doi.org/10.1145/335191.335372 Han et al., Mining frequent patterns without candidate + * generation]]. * * @param minSupport the minimal support level of the frequent pattern, any pattern appears * more than (minSupport * size-of-the-dataset) times will be output * @param numPartitions number of partitions used by parallel FP-growth + * + * @see [[http://en.wikipedia.org/wiki/Association_rule_learning Association rule learning + * (Wikipedia)]] */ +@Experimental class FPGrowth private ( private var minSupport: Double, private var numPartitions: Int) extends Logging with Serializable { /** - * Constructs a FPGrowth instance with default parameters: - * {minSupport: 0.3, numPartitions: auto} + * Constructs a default instance with default parameters {minSupport: `0.3`, numPartitions: same + * as the input data}. */ def this() = this(0.3, -1) /** - * Sets the minimal support level (default: 0.3). + * Sets the minimal support level (default: `0.3`). */ def setMinSupport(minSupport: Double): this.type = { this.minSupport = minSupport 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 ad7e86827b368..84f8ac2e0d9cd 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 @@ -115,7 +115,7 @@ sealed trait Matrix extends Serializable { * * @param numRows number of rows * @param numCols number of columns - * @param values matrix entries in column major + * @param values matrix entries in column major if not transposed or in row major otherwise * @param isTransposed whether the matrix is transposed. If true, `values` stores the matrix in * row major. */ @@ -187,7 +187,7 @@ class DenseMatrix( this } - override def transpose: Matrix = new DenseMatrix(numCols, numRows, values, !isTransposed) + override def transpose: DenseMatrix = new DenseMatrix(numCols, numRows, values, !isTransposed) private[spark] override def foreachActive(f: (Int, Int, Double) => Unit): Unit = { if (!isTransposed) { @@ -217,9 +217,11 @@ class DenseMatrix( } } - /** Generate a `SparseMatrix` from the given `DenseMatrix`. The new matrix will have isTransposed - * set to false. */ - def toSparse(): SparseMatrix = { + /** + * Generate a `SparseMatrix` from the given `DenseMatrix`. The new matrix will have isTransposed + * set to false. + */ + def toSparse: SparseMatrix = { val spVals: MArrayBuilder[Double] = new MArrayBuilder.ofDouble val colPtrs: Array[Int] = new Array[Int](numCols + 1) val rowIndices: MArrayBuilder[Int] = new MArrayBuilder.ofInt @@ -282,7 +284,7 @@ object DenseMatrix { } /** - * Generate a `DenseMatrix` consisting of i.i.d. uniform random numbers. + * Generate a `DenseMatrix` consisting of `i.i.d.` uniform random numbers. * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix * @param rng a random number generator @@ -293,7 +295,7 @@ object DenseMatrix { } /** - * Generate a `DenseMatrix` consisting of i.i.d. gaussian random numbers. + * Generate a `DenseMatrix` consisting of `i.i.d.` gaussian random numbers. * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix * @param rng a random number generator @@ -336,10 +338,10 @@ object DenseMatrix { * * @param numRows number of rows * @param numCols number of columns - * @param colPtrs the index corresponding to the start of a new column - * @param rowIndices the row index of the entry. They must be in strictly increasing order for each - * column - * @param values non-zero matrix entries in column major + * @param colPtrs the index corresponding to the start of a new column (if not transposed) + * @param rowIndices the row index of the entry (if not transposed). They must be in strictly + * increasing order for each column + * @param values nonzero matrix entries in column major (if not transposed) * @param isTransposed whether the matrix is transposed. If true, the matrix can be considered * Compressed Sparse Row (CSR) format, where `colPtrs` behaves as rowPtrs, * and `rowIndices` behave as colIndices, and `values` are stored in row major. @@ -434,7 +436,7 @@ class SparseMatrix( this } - override def transpose: Matrix = + override def transpose: SparseMatrix = new SparseMatrix(numCols, numRows, colPtrs, rowIndices, values, !isTransposed) private[spark] override def foreachActive(f: (Int, Int, Double) => Unit): Unit = { @@ -464,9 +466,11 @@ class SparseMatrix( } } - /** Generate a `DenseMatrix` from the given `SparseMatrix`. The new matrix will have isTransposed - * set to false. */ - def toDense(): DenseMatrix = { + /** + * Generate a `DenseMatrix` from the given `SparseMatrix`. The new matrix will have isTransposed + * set to false. + */ + def toDense: DenseMatrix = { new DenseMatrix(numRows, numCols, toArray) } } @@ -593,7 +597,7 @@ object SparseMatrix { } /** - * Generate a `SparseMatrix` consisting of i.i.d. uniform random numbers. The number of non-zero + * Generate a `SparseMatrix` consisting of `i.i.d`. uniform random numbers. The number of non-zero * elements equal the ceiling of `numRows` x `numCols` x `density` * * @param numRows number of rows of the matrix @@ -608,7 +612,7 @@ object SparseMatrix { } /** - * Generate a `SparseMatrix` consisting of i.i.d. gaussian random numbers. + * Generate a `SparseMatrix` consisting of `i.i.d`. gaussian random numbers. * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix * @param density the desired density for the matrix @@ -626,7 +630,7 @@ object SparseMatrix { * @return Square `SparseMatrix` with size `values.length` x `values.length` and non-zero * `values` on the diagonal */ - def diag(vector: Vector): SparseMatrix = { + def spdiag(vector: Vector): SparseMatrix = { val n = vector.size vector match { case sVec: SparseVector => @@ -722,7 +726,7 @@ object Matrices { def speye(n: Int): Matrix = SparseMatrix.speye(n) /** - * Generate a `DenseMatrix` consisting of i.i.d. uniform random numbers. + * Generate a `DenseMatrix` consisting of `i.i.d.` uniform random numbers. * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix * @param rng a random number generator @@ -732,7 +736,7 @@ object Matrices { DenseMatrix.rand(numRows, numCols, rng) /** - * Generate a `SparseMatrix` consisting of i.i.d. gaussian random numbers. + * Generate a `SparseMatrix` consisting of `i.i.d.` gaussian random numbers. * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix * @param density the desired density for the matrix @@ -743,7 +747,7 @@ object Matrices { SparseMatrix.sprand(numRows, numCols, density, rng) /** - * Generate a `DenseMatrix` consisting of i.i.d. gaussian random numbers. + * Generate a `DenseMatrix` consisting of `i.i.d.` gaussian random numbers. * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix * @param rng a random number generator @@ -753,7 +757,7 @@ object Matrices { DenseMatrix.randn(numRows, numCols, rng) /** - * Generate a `SparseMatrix` consisting of i.i.d. gaussian random numbers. + * Generate a `SparseMatrix` consisting of `i.i.d.` gaussian random numbers. * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix * @param density the desired density for the matrix 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 8f75e6f46e05d..77785bdbd03d9 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 @@ -244,8 +244,7 @@ object Vectors { } /** - * Parses a string resulted from `Vector#toString` into - * an [[org.apache.spark.mllib.linalg.Vector]]. + * Parses a string resulted from [[Vector.toString]] into a [[Vector]]. */ def parse(s: String): Vector = { parseNumeric(NumericParser.parse(s)) @@ -483,6 +482,7 @@ class DenseVector(val values: Array[Double]) extends Vector { } object DenseVector { + /** Extracts the value array from a dense vector. */ def unapply(dv: DenseVector): Option[Array[Double]] = Some(dv.values) } 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 3871152d065a7..1d253963130f1 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 @@ -21,7 +21,8 @@ import scala.collection.mutable.ArrayBuffer import breeze.linalg.{DenseMatrix => BDM} -import org.apache.spark.{SparkException, Logging, Partitioner} +import org.apache.spark.{Logging, Partitioner, SparkException} +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices, Matrix, SparseMatrix} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -104,6 +105,8 @@ private[mllib] object GridPartitioner { } /** + * :: Experimental :: + * * Represents a distributed matrix in blocks of local matrices. * * @param blocks The RDD of sub-matrix blocks ((blockRowIndex, blockColIndex), sub-matrix) that @@ -118,6 +121,7 @@ private[mllib] object GridPartitioner { * @param nCols Number of columns of this matrix. If the supplied value is less than or equal to * zero, the number of columns will be calculated when `numCols` is invoked. */ +@Experimental class BlockMatrix( val blocks: RDD[((Int, Int), Matrix)], val rowsPerBlock: Int, @@ -177,6 +181,10 @@ class BlockMatrix( assert(cols <= nCols, s"The number of columns $cols is more than claimed $nCols.") } + /** + * Validates the block matrix info against the matrix data (`blocks`) and throws an exception if + * any error is found. + */ def validate(): Unit = { logDebug("Validating BlockMatrix...") // check if the matrix is larger than the claimed dimensions @@ -351,7 +359,7 @@ class BlockMatrix( if (a.nonEmpty && b.nonEmpty) { val C = b.head match { case dense: DenseMatrix => a.head.multiply(dense) - case sparse: SparseMatrix => a.head.multiply(sparse.toDense()) + case sparse: SparseMatrix => a.head.multiply(sparse.toDense) case _ => throw new SparkException(s"Unrecognized matrix type ${b.head.getClass}.") } Iterator(((blockRowIndex, blockColIndex), C.toBreeze)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala index 955c593a085d5..8341bb86afd71 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala @@ -29,13 +29,13 @@ import org.apache.spark.util.Utils /** * :: Experimental :: - * Generator methods for creating RDDs comprised of i.i.d. samples from some distribution. + * Generator methods for creating RDDs comprised of `i.i.d.` samples from some distribution. */ @Experimental object RandomRDDs { /** - * Generates an RDD comprised of i.i.d. samples from the uniform distribution `U(0.0, 1.0)`. + * Generates an RDD comprised of `i.i.d.` samples from the uniform distribution `U(0.0, 1.0)`. * * To transform the distribution in the generated RDD from `U(0.0, 1.0)` to `U(a, b)`, use * `RandomRDDs.uniformRDD(sc, n, p, seed).map(v => a + (b - a) * v)`. @@ -44,7 +44,7 @@ object RandomRDDs { * @param size Size of the RDD. * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). * @param seed Random seed (default: a random long integer). - * @return RDD[Double] comprised of i.i.d. samples ~ `U(0.0, 1.0)`. + * @return RDD[Double] comprised of `i.i.d.` samples ~ `U(0.0, 1.0)`. */ def uniformRDD( sc: SparkContext, @@ -81,7 +81,7 @@ object RandomRDDs { } /** - * Generates an RDD comprised of i.i.d. samples from the standard normal distribution. + * Generates an RDD comprised of `i.i.d.` samples from the standard normal distribution. * * To transform the distribution in the generated RDD from standard normal to some other normal * `N(mean, sigma^2^)`, use `RandomRDDs.normalRDD(sc, n, p, seed).map(v => mean + sigma * v)`. @@ -90,7 +90,7 @@ object RandomRDDs { * @param size Size of the RDD. * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). * @param seed Random seed (default: a random long integer). - * @return RDD[Double] comprised of i.i.d. samples ~ N(0.0, 1.0). + * @return RDD[Double] comprised of `i.i.d.` samples ~ N(0.0, 1.0). */ def normalRDD( sc: SparkContext, @@ -127,14 +127,15 @@ object RandomRDDs { } /** - * Generates an RDD comprised of i.i.d. samples from the Poisson distribution with the input mean. + * Generates an RDD comprised of `i.i.d.` samples from the Poisson distribution with the input + * mean. * * @param sc SparkContext used to create the RDD. * @param mean Mean, or lambda, for the Poisson distribution. * @param size Size of the RDD. * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). * @param seed Random seed (default: a random long integer). - * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + * @return RDD[Double] comprised of `i.i.d.` samples ~ Pois(mean). */ def poissonRDD( sc: SparkContext, @@ -177,7 +178,7 @@ object RandomRDDs { } /** - * Generates an RDD comprised of i.i.d. samples from the exponential distribution with + * Generates an RDD comprised of `i.i.d.` samples from the exponential distribution with * the input mean. * * @param sc SparkContext used to create the RDD. @@ -185,7 +186,7 @@ object RandomRDDs { * @param size Size of the RDD. * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). * @param seed Random seed (default: a random long integer). - * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + * @return RDD[Double] comprised of `i.i.d.` samples ~ Pois(mean). */ def exponentialRDD( sc: SparkContext, @@ -228,7 +229,7 @@ object RandomRDDs { } /** - * Generates an RDD comprised of i.i.d. samples from the gamma distribution with the input + * Generates an RDD comprised of `i.i.d.` samples from the gamma distribution with the input * shape and scale. * * @param sc SparkContext used to create the RDD. @@ -237,7 +238,7 @@ object RandomRDDs { * @param size Size of the RDD. * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). * @param seed Random seed (default: a random long integer). - * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + * @return RDD[Double] comprised of `i.i.d.` samples ~ Pois(mean). */ def gammaRDD( sc: SparkContext, @@ -287,7 +288,7 @@ object RandomRDDs { } /** - * Generates an RDD comprised of i.i.d. samples from the log normal distribution with the input + * Generates an RDD comprised of `i.i.d.` samples from the log normal distribution with the input * mean and standard deviation * * @param sc SparkContext used to create the RDD. @@ -296,7 +297,7 @@ object RandomRDDs { * @param size Size of the RDD. * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). * @param seed Random seed (default: a random long integer). - * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + * @return RDD[Double] comprised of `i.i.d.` samples ~ Pois(mean). */ def logNormalRDD( sc: SparkContext, @@ -348,14 +349,14 @@ object RandomRDDs { /** * :: DeveloperApi :: - * Generates an RDD comprised of i.i.d. samples produced by the input RandomDataGenerator. + * Generates an RDD comprised of `i.i.d.` samples produced by the input RandomDataGenerator. * * @param sc SparkContext used to create the RDD. * @param generator RandomDataGenerator used to populate the RDD. * @param size Size of the RDD. * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). * @param seed Random seed (default: a random long integer). - * @return RDD[Double] comprised of i.i.d. samples produced by generator. + * @return RDD[Double] comprised of `i.i.d.` samples produced by generator. */ @DeveloperApi def randomRDD[T: ClassTag]( @@ -370,7 +371,7 @@ object RandomRDDs { // TODO Generate RDD[Vector] from multivariate distributions. /** - * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the + * Generates an RDD[Vector] with vectors containing `i.i.d.` samples drawn from the * uniform distribution on `U(0.0, 1.0)`. * * @param sc SparkContext used to create the RDD. @@ -424,7 +425,7 @@ object RandomRDDs { } /** - * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the + * Generates an RDD[Vector] with vectors containing `i.i.d.` samples drawn from the * standard normal distribution. * * @param sc SparkContext used to create the RDD. @@ -432,7 +433,7 @@ object RandomRDDs { * @param numCols Number of elements in each Vector. * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). * @param seed Random seed (default: a random long integer). - * @return RDD[Vector] with vectors containing i.i.d. samples ~ `N(0.0, 1.0)`. + * @return RDD[Vector] with vectors containing `i.i.d.` samples ~ `N(0.0, 1.0)`. */ def normalVectorRDD( sc: SparkContext, @@ -478,7 +479,7 @@ object RandomRDDs { } /** - * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from a + * Generates an RDD[Vector] with vectors containing `i.i.d.` samples drawn from a * log normal distribution. * * @param sc SparkContext used to create the RDD. @@ -488,7 +489,7 @@ object RandomRDDs { * @param numCols Number of elements in each Vector. * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). * @param seed Random seed (default: a random long integer). - * @return RDD[Vector] with vectors containing i.i.d. samples. + * @return RDD[Vector] with vectors containing `i.i.d.` samples. */ def logNormalVectorRDD( sc: SparkContext, @@ -544,7 +545,7 @@ object RandomRDDs { } /** - * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the + * Generates an RDD[Vector] with vectors containing `i.i.d.` samples drawn from the * Poisson distribution with the input mean. * * @param sc SparkContext used to create the RDD. @@ -553,7 +554,7 @@ object RandomRDDs { * @param numCols Number of elements in each Vector. * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`) * @param seed Random seed (default: a random long integer). - * @return RDD[Vector] with vectors containing i.i.d. samples ~ Pois(mean). + * @return RDD[Vector] with vectors containing `i.i.d.` samples ~ Pois(mean). */ def poissonVectorRDD( sc: SparkContext, @@ -603,7 +604,7 @@ object RandomRDDs { } /** - * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the + * Generates an RDD[Vector] with vectors containing `i.i.d.` samples drawn from the * exponential distribution with the input mean. * * @param sc SparkContext used to create the RDD. @@ -612,7 +613,7 @@ object RandomRDDs { * @param numCols Number of elements in each Vector. * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`) * @param seed Random seed (default: a random long integer). - * @return RDD[Vector] with vectors containing i.i.d. samples ~ Exp(mean). + * @return RDD[Vector] with vectors containing `i.i.d.` samples ~ Exp(mean). */ def exponentialVectorRDD( sc: SparkContext, @@ -665,7 +666,7 @@ object RandomRDDs { /** - * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the + * Generates an RDD[Vector] with vectors containing `i.i.d.` samples drawn from the * gamma distribution with the input shape and scale. * * @param sc SparkContext used to create the RDD. @@ -675,7 +676,7 @@ object RandomRDDs { * @param numCols Number of elements in each Vector. * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`) * @param seed Random seed (default: a random long integer). - * @return RDD[Vector] with vectors containing i.i.d. samples ~ Exp(mean). + * @return RDD[Vector] with vectors containing `i.i.d.` samples ~ Exp(mean). */ def gammaVectorRDD( sc: SparkContext, @@ -731,7 +732,7 @@ object RandomRDDs { /** * :: DeveloperApi :: - * Generates an RDD[Vector] with vectors containing i.i.d. samples produced by the + * Generates an RDD[Vector] with vectors containing `i.i.d.` samples produced by the * input RandomDataGenerator. * * @param sc SparkContext used to create the RDD. @@ -740,7 +741,7 @@ object RandomRDDs { * @param numCols Number of elements in each Vector. * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). * @param seed Random seed (default: a random long integer). - * @return RDD[Vector] with vectors containing i.i.d. samples produced by generator. + * @return RDD[Vector] with vectors containing `i.i.d.` samples produced by generator. */ @DeveloperApi def randomVectorRDD(sc: SparkContext, 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 f4f51f2ac5210..4bb28d1b1e071 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 @@ -25,10 +25,8 @@ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel /** - * :: Experimental :: * A more compact class to represent a rating than Tuple3[Int, Int, Double]. */ -@Experimental case class Rating(user: Int, product: Int, rating: Double) /** @@ -135,10 +133,8 @@ class ALS private ( } /** - * :: Experimental :: * Sets the constant used in computing confidence in implicit ALS. Default: 1.0. */ - @Experimental def setAlpha(alpha: Double): this.type = { this.alpha = alpha this diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala index 5ed6477bae3b2..cb70852e3cc8d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala @@ -23,10 +23,13 @@ import java.util.Arrays.binarySearch import scala.collection.mutable.ArrayBuffer +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD} import org.apache.spark.rdd.RDD /** + * :: Experimental :: + * * Regression model for isotonic regression. * * @param boundaries Array of boundaries for which predictions are known. @@ -35,6 +38,7 @@ import org.apache.spark.rdd.RDD * Results of isotonic regression and therefore monotone. * @param isotonic indicates whether this is isotonic or antitonic. */ +@Experimental class IsotonicRegressionModel ( val boundaries: Array[Double], val predictions: Array[Double], @@ -123,6 +127,8 @@ class IsotonicRegressionModel ( } /** + * :: Experimental :: + * * Isotonic regression. * Currently implemented using parallelized pool adjacent violators algorithm. * Only univariate (single feature) algorithm supported. @@ -130,14 +136,17 @@ class IsotonicRegressionModel ( * Sequential PAV implementation based on: * Tibshirani, Ryan J., Holger Hoefling, and Robert Tibshirani. * "Nearly-isotonic regression." Technometrics 53.1 (2011): 54-61. - * Available from http://www.stat.cmu.edu/~ryantibs/papers/neariso.pdf + * Available from [[http://www.stat.cmu.edu/~ryantibs/papers/neariso.pdf]] * * Sequential PAV parallelization based on: * Kearsley, Anthony J., Richard A. Tapia, and Michael W. Trosset. * "An approach to parallelizing isotonic regression." * Applied Mathematics and Parallel Computing. Physica-Verlag HD, 1996. 141-147. - * Available from http://softlib.rice.edu/pub/CRPC-TRs/reports/CRPC-TR96640.pdf + * Available from [[http://softlib.rice.edu/pub/CRPC-TRs/reports/CRPC-TR96640.pdf]] + * + * @see [[http://en.wikipedia.org/wiki/Isotonic_regression Isotonic regression (Wikipedia)]] */ +@Experimental class IsotonicRegression private (private var isotonic: Boolean) extends Serializable { /** 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 44a8dbb994cfb..c854f124451cf 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 @@ -39,14 +39,14 @@ import org.apache.spark.streaming.dstream.DStream * * For example usage, see `StreamingLinearRegressionWithSGD`. * - * NOTE(Freeman): In some use cases, the order in which trainOn and predictOn + * NOTE: In some use cases, the order in which trainOn and predictOn * are called in an application will affect the results. When called on * the same DStream, if trainOn is called before predictOn, when new data * arrive the model will update and the prediction will be based on the new * model. Whereas if predictOn is called first, the prediction will use the model * from the previous update. * - * NOTE(Freeman): It is ok to call predictOn repeatedly on multiple streams; this + * NOTE: It is ok to call predictOn repeatedly on multiple streams; this * will generate predictions for each one all using the current model. * It is also ok to call trainOn on different streams; this will update * the model using each of the different sources, in sequence. 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 3cf4e807b4cf7..b3fad0c52d655 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 @@ -26,36 +26,32 @@ import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult} import org.apache.spark.rdd.RDD /** + * :: Experimental :: * API for statistical functions in MLlib. */ @Experimental object Statistics { /** - * :: Experimental :: * Computes column-wise summary statistics for the input RDD[Vector]. * * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. * @return [[MultivariateStatisticalSummary]] object containing column-wise summary statistics. */ - @Experimental def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { new RowMatrix(X).computeColumnSummaryStatistics() } /** - * :: Experimental :: * Compute the Pearson correlation matrix for the input RDD of Vectors. * Columns with 0 covariance produce NaN entries in the correlation matrix. * * @param X an RDD[Vector] for which the correlation matrix is to be computed. * @return Pearson correlation matrix comparing columns in X. */ - @Experimental def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) /** - * :: Experimental :: * Compute the correlation matrix for the input RDD of Vectors using the specified method. * Methods currently supported: `pearson` (default), `spearman`. * @@ -69,11 +65,9 @@ object Statistics { * Supported: `pearson` (default), `spearman` * @return Correlation matrix comparing columns in X. */ - @Experimental def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) /** - * :: Experimental :: * Compute the Pearson correlation for the input RDDs. * Returns NaN if either vector has 0 variance. * @@ -84,11 +78,9 @@ object Statistics { * @param y RDD[Double] of the same cardinality as x. * @return A Double containing the Pearson correlation between the two input RDD[Double]s */ - @Experimental def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) /** - * :: Experimental :: * Compute the correlation for the input RDDs using the specified method. * Methods currently supported: `pearson` (default), `spearman`. * @@ -99,14 +91,12 @@ object Statistics { * @param y RDD[Double] of the same cardinality as x. * @param method String specifying the method to use for computing correlation. * Supported: `pearson` (default), `spearman` - *@return A Double containing the correlation between the two input RDD[Double]s using the + * @return A Double containing the correlation between the two input RDD[Double]s using the * specified method. */ - @Experimental def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) /** - * :: Experimental :: * Conduct Pearson's chi-squared goodness of fit test of the observed data against the * expected distribution. * @@ -120,13 +110,11 @@ object Statistics { * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, * the method used, and the null hypothesis. */ - @Experimental def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { ChiSqTest.chiSquared(observed, expected) } /** - * :: Experimental :: * Conduct Pearson's chi-squared goodness of fit test of the observed data against the uniform * distribution, with each category having an expected frequency of `1 / observed.size`. * @@ -136,11 +124,9 @@ object Statistics { * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, * the method used, and the null hypothesis. */ - @Experimental def chiSqTest(observed: Vector): ChiSqTestResult = ChiSqTest.chiSquared(observed) /** - * :: Experimental :: * Conduct Pearson's independence test on the input contingency matrix, which cannot contain * negative entries or columns or rows that sum up to 0. * @@ -148,11 +134,9 @@ object Statistics { * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, * the method used, and the null hypothesis. */ - @Experimental def chiSqTest(observed: Matrix): ChiSqTestResult = ChiSqTest.chiSquaredMatrix(observed) /** - * :: Experimental :: * Conduct Pearson's independence test for every feature against the label across the input RDD. * For each feature, the (feature, label) pairs are converted into a contingency matrix for which * the chi-squared statistic is computed. All label and feature values must be categorical. @@ -162,7 +146,6 @@ object Statistics { * @return an array containing the ChiSquaredTestResult for every feature against the label. * The order of the elements in the returned array reflects the order of input features. */ - @Experimental def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = { ChiSqTest.chiSquaredFeatures(data) } diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java index 704d484d0b585..3349c5022423a 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java @@ -71,8 +71,8 @@ public void diagonalMatrixConstruction() { Matrix sm = Matrices.diag(sv); DenseMatrix d = DenseMatrix.diag(v); DenseMatrix sd = DenseMatrix.diag(sv); - SparseMatrix s = SparseMatrix.diag(v); - SparseMatrix ss = SparseMatrix.diag(sv); + SparseMatrix s = SparseMatrix.spdiag(v); + SparseMatrix ss = SparseMatrix.spdiag(sv); assertArrayEquals(m.toArray(), sm.toArray(), 0.0); assertArrayEquals(d.toArray(), sm.toArray(), 0.0); 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 b1ebfde0e5e57..c098b5458fe6b 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 @@ -137,8 +137,8 @@ class MatricesSuite extends FunSuite { val spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values) val deMat1 = new DenseMatrix(m, n, allValues) - val spMat2 = deMat1.toSparse() - val deMat2 = spMat1.toDense() + val spMat2 = deMat1.toSparse + val deMat2 = spMat1.toDense assert(spMat1.toBreeze === spMat2.toBreeze) assert(deMat1.toBreeze === deMat2.toBreeze) @@ -185,8 +185,8 @@ class MatricesSuite extends FunSuite { assert(!dA.toArray.eq(dAT.toArray), "has to have a new array") assert(dA.values.eq(dAT.transpose.asInstanceOf[DenseMatrix].values), "should not copy array") - assert(dAT.toSparse().toBreeze === sATexpected.toBreeze) - assert(sAT.toDense().toBreeze === dATexpected.toBreeze) + assert(dAT.toSparse.toBreeze === sATexpected.toBreeze) + assert(sAT.toDense.toBreeze === dATexpected.toBreeze) } test("foreachActive") { From 9d3a75ef80d0b736d1366a464bf00b64a120f461 Mon Sep 17 00:00:00 2001 From: q00251598 Date: Wed, 4 Feb 2015 23:16:01 -0800 Subject: [PATCH 086/578] [SPARK-5606][SQL] Support plus sign in HiveContext Now spark version is only support ```SELECT -key FROM DECIMAL_UDF;``` in HiveContext. This patch is used to support ```SELECT +key FROM DECIMAL_UDF;``` in HiveContext. Author: q00251598 Closes #4378 from watermen/SPARK-5606 and squashes the following commits: 777f132 [q00251598] sql-case22 74dd368 [q00251598] sql-case22 1a67410 [q00251598] sql-case22 c5cd5bc [q00251598] sql-case22 --- 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 286d13944c892..62e9d92eac076 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 @@ -1103,6 +1103,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) => UnaryMinus(nodeToExpr(child)) case Token("~", child :: Nil) => BitwiseNot(nodeToExpr(child)) case Token("+", left :: right:: Nil) => Add(nodeToExpr(left), nodeToExpr(right)) From 7d789e117d6ddaf66159e708db600f2d8db8d787 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 4 Feb 2015 23:44:34 -0800 Subject: [PATCH 087/578] [SPARK-5612][SQL] Move DataFrame implicit functions into SQLContext.implicits. Author: Reynold Xin Closes #4386 from rxin/df-implicits and squashes the following commits: 9d96606 [Reynold Xin] style fix edd296b [Reynold Xin] ReplSuite 1c946ab [Reynold Xin] [SPARK-5612][SQL] Move DataFrame implicit functions into SQLContext.implicits. --- .../spark/examples/ml/CrossValidatorExample.scala | 6 +++--- .../org/apache/spark/examples/ml/MovieLensALS.scala | 2 +- .../apache/spark/examples/ml/SimpleParamsExample.scala | 6 +++--- .../examples/ml/SimpleTextClassificationPipeline.scala | 6 +++--- .../apache/spark/examples/mllib/DatasetExample.scala | 2 +- .../org/apache/spark/examples/sql/RDDRelation.scala | 10 +++++----- .../apache/spark/examples/sql/hive/HiveFromSpark.scala | 3 ++- .../scala/org/apache/spark/ml/recommendation/ALS.scala | 2 +- .../org/apache/spark/ml/recommendation/ALSSuite.scala | 2 +- .../test/scala/org/apache/spark/repl/ReplSuite.scala | 4 ++-- .../main/scala/org/apache/spark/sql/SQLContext.scala | 4 ++-- .../org/apache/spark/sql/parquet/ParquetTest.scala | 1 + .../scala/org/apache/spark/sql/CachedTableSuite.scala | 2 ++ .../scala/org/apache/spark/sql/DataFrameSuite.scala | 3 ++- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 2 ++ .../spark/sql/ScalaReflectionRelationSuite.scala | 3 +++ .../src/test/scala/org/apache/spark/sql/TestData.scala | 3 +-- .../org/apache/spark/sql/UserDefinedTypeSuite.scala | 3 ++- .../sql/columnar/InMemoryColumnarQuerySuite.scala | 2 ++ .../sql/columnar/PartitionBatchPruningSuite.scala | 2 ++ .../org/apache/spark/sql/parquet/ParquetIOSuite.scala | 2 ++ .../spark/sql/hive/InsertIntoHiveTableSuite.scala | 2 ++ .../spark/sql/hive/MetastoreDataSourcesSuite.scala | 3 +++ .../spark/sql/hive/execution/HiveQuerySuite.scala | 2 ++ .../spark/sql/hive/execution/HiveResolutionSuite.scala | 3 ++- .../apache/spark/sql/hive/execution/HiveUdfSuite.scala | 4 +++- .../spark/sql/hive/execution/SQLQuerySuite.scala | 3 +++ .../org/apache/spark/sql/parquet/parquetSuites.scala | 2 ++ 28 files changed, 60 insertions(+), 29 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 283bb80f1c788..b6c30a007d88f 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 @@ -43,10 +43,10 @@ object CrossValidatorExample { 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), @@ -92,7 +92,7 @@ object CrossValidatorExample { val cvModel = crossval.fit(training) // 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"), 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 b7885829459a3..836ea2e01201e 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 @@ -109,7 +109,7 @@ object MovieLensALS { val conf = new SparkConf().setAppName(s"MovieLensALS with $params") val sc = new SparkContext(conf) val sqlContext = new SQLContext(sc) - import sqlContext._ + import sqlContext.implicits._ val ratings = sc.textFile(params.ratings).map(Rating.parseRating).cache() 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 95cc9801eaeb9..4d1530cd1349f 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 @@ -37,12 +37,12 @@ object SimpleParamsExample { 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 Java Beans // into DataFrames, where it uses the bean metadata to infer the schema. - val training = sparkContext.parallelize(Seq( + 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)), @@ -81,7 +81,7 @@ object SimpleParamsExample { println("Model 2 was fit using parameters: " + model2.fittingParamMap) // Prepare test documents. - val test = sparkContext.parallelize(Seq( + 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)))) 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 065db62b0f5ed..dbbe01dd5ce8e 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 @@ -44,10 +44,10 @@ object SimpleTextClassificationPipeline { 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), @@ -71,7 +71,7 @@ object SimpleTextClassificationPipeline { val model = pipeline.fit(training) // 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"), 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 ab58375649d25..89b6255991a38 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 @@ -71,7 +71,7 @@ object DatasetExample { val conf = new SparkConf().setAppName(s"DatasetExample with $params") val sc = new SparkContext(conf) val sqlContext = new SQLContext(sc) - import sqlContext._ // for implicit conversions + import sqlContext.implicits._ // for implicit conversions // Load input data val origData: RDD[LabeledPoint] = params.dataFormat match { 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 82a0b637b3cff..1eac3c8d03e39 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 @@ -32,7 +32,7 @@ object RDDRelation { val sqlContext = new SQLContext(sc) // Importing the SQL context gives access to all the SQL functions and implicit conversions. - import sqlContext._ + import sqlContext.implicits._ val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) // Any RDD containing case classes can be registered as a table. The schema of the table is @@ -41,15 +41,15 @@ object RDDRelation { // Once tables have been registered, you can run SQL queries over them. println("Result of SELECT *:") - sql("SELECT * FROM records").collect().foreach(println) + sqlContext.sql("SELECT * FROM records").collect().foreach(println) // Aggregation queries are also supported. - val count = sql("SELECT COUNT(*) FROM records").collect().head.getLong(0) + val count = sqlContext.sql("SELECT COUNT(*) FROM records").collect().head.getLong(0) println(s"COUNT(*): $count") // The results of SQL queries are themselves RDDs and support all normal RDD functions. The // items in the RDD are of type Row, which allows you to access each column by ordinal. - val rddFromSql = sql("SELECT key, value FROM records WHERE key < 10") + val rddFromSql = sqlContext.sql("SELECT key, value FROM records WHERE key < 10") println("Result of RDD.map:") rddFromSql.map(row => s"Key: ${row(0)}, Value: ${row(1)}").collect().foreach(println) @@ -68,7 +68,7 @@ object RDDRelation { // These files can also be registered as tables. parquetFile.registerTempTable("parquetFile") - sql("SELECT * FROM parquetFile").collect().foreach(println) + sqlContext.sql("SELECT * FROM parquetFile").collect().foreach(println) sc.stop() } 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 5725da1848114..15754cdfcc35e 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 @@ -43,7 +43,8 @@ object HiveFromSpark { // HiveContext. When not configured by the hive-site.xml, the context automatically // creates metastore_db and warehouse in the current directory. val hiveContext = new HiveContext(sc) - import hiveContext._ + import hiveContext.implicits._ + import hiveContext.sql sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") sql(s"LOAD DATA LOCAL INPATH '${kv1File.getAbsolutePath}' INTO TABLE src") 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 c7bec7a845a11..09456b289e2dc 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 @@ -122,7 +122,7 @@ class ALSModel private[ml] ( def setPredictionCol(value: String): this.type = set(predictionCol, value) override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { - import dataset.sqlContext.createDataFrame + import dataset.sqlContext.implicits._ val map = this.paramMap ++ paramMap val users = userFactors.toDataFrame("id", "features") val items = itemFactors.toDataFrame("id", "features") 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 acc447742bad0..cb7d57de35c34 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 @@ -350,7 +350,7 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext with Logging { numItemBlocks: Int = 3, targetRMSE: Double = 0.05): Unit = { val sqlContext = this.sqlContext - import sqlContext.createDataFrame + import sqlContext.implicits._ val als = new ALS() .setRank(rank) .setRegParam(regParam) 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 e594ad868ea1c..201f2672d5474 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 @@ -255,12 +255,12 @@ class ReplSuite extends FunSuite { assertDoesNotContain("Exception", output) } - test("SPARK-2576 importing SQLContext.createDataFrame.") { + test("SPARK-2576 importing SQLContext.implicits._") { // 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.createDataFrame + |import sqlContext.implicits._ |case class TestCaseClass(value: Int) |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toDataFrame.collect() """.stripMargin) 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 5ab5494f8042f..01620aa0acd49 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 @@ -166,6 +166,7 @@ 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. */ object implicits { @@ -192,8 +193,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * * @group userf */ - // TODO: Remove implicit here. - implicit def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { + def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { SparkPlan.currentContext.set(self) val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] val attributeSeq = schema.toAttributes 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 9d6c529574da0..8d3e094e3344d 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 @@ -88,6 +88,7 @@ trait ParquetTest { protected def withParquetFile[T <: Product: ClassTag: TypeTag] (data: Seq[T]) (f: String => Unit): Unit = { + import sqlContext.implicits._ withTempPath { file => sparkContext.parallelize(data).saveAsParquetFile(file.getCanonicalPath) f(file.getCanonicalPath) 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 acb5677c4b77b..1318750a4a3b0 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 @@ -34,6 +34,8 @@ 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 { 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 e588555ad01d4..74c29459d2e47 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,8 @@ import org.apache.spark.sql.Dsl._ import org.apache.spark.sql.types._ /* Implicits */ -import org.apache.spark.sql.test.TestSQLContext.{createDataFrame, logicalPlanToSparkQuery} +import org.apache.spark.sql.test.TestSQLContext.logicalPlanToSparkQuery +import org.apache.spark.sql.test.TestSQLContext.implicits._ import scala.language.postfixOps 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 8f3d4265a286d..dc8ee41712fcd 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 @@ -35,6 +35,8 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { // Make sure the tables are loaded. TestData + import org.apache.spark.sql.test.TestSQLContext.implicits._ + var origZone: TimeZone = _ override protected def beforeAll() { origZone = TimeZone.getDefault 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 f26fcc0385b68..93782619826f0 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 @@ -75,6 +75,9 @@ case class ComplexReflectData( dataField: Data) class ScalaReflectionRelationSuite extends FunSuite { + + import org.apache.spark.sql.test.TestSQLContext.implicits._ + 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)) 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 dd781169ca57f..0ed437edd05fd 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 @@ -22,9 +22,8 @@ import java.sql.Timestamp import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.Dsl._ import org.apache.spark.sql.test._ +import org.apache.spark.sql.test.TestSQLContext.implicits._ -/* Implicits */ -import org.apache.spark.sql.test.TestSQLContext._ case class TestData(key: Int, value: 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 117a51173481d..3c1657cd5fc3a 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,7 +22,8 @@ import scala.beans.{BeanInfo, BeanProperty} import org.apache.spark.rdd.RDD import org.apache.spark.sql.Dsl._ import org.apache.spark.sql.test.TestSQLContext -import org.apache.spark.sql.test.TestSQLContext.{udf => _, _} +import org.apache.spark.sql.test.TestSQLContext.{sparkContext, sql} +import org.apache.spark.sql.test.TestSQLContext.implicits._ import org.apache.spark.sql.types._ 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 3d33484ab0eb9..86b1b5fda1c0f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -28,6 +28,8 @@ 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) 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 fe9a69edbb920..55a9f735b3506 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 @@ -26,6 +26,8 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be 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") 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 d9ab16baf9a66..0bc246c645602 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 @@ -98,6 +98,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest { } test("fixed-length decimals") { + import org.apache.spark.sql.test.TestSQLContext.implicits._ + def makeDecimalRDD(decimal: DecimalType): DataFrame = sparkContext .parallelize(0 to 1000) 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 4dd96bd5a1b77..869d01eb398c5 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 @@ -30,6 +30,8 @@ import org.apache.spark.sql.hive.test.TestHive._ case class TestData(key: Int, value: String) class InsertIntoHiveTableSuite extends QueryTest { + 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") 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 85795acb658e2..c23575fe96898 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 @@ -36,6 +36,9 @@ import org.apache.spark.sql.hive.test.TestHive._ * 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 (ctasPath.exists()) Utils.deleteRecursively(ctasPath) 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 a321452cef74f..407d6058c33ed 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 @@ -43,6 +43,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { private val originalTimeZone = TimeZone.getDefault private val originalLocale = Locale.getDefault + import org.apache.spark.sql.hive.test.TestHive.implicits._ + override def beforeAll() { TestHive.cacheTables = true // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) 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 422e843d2b0d2..178ece447758e 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 @@ -18,7 +18,8 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.hive.test.TestHive -import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.hive.test.TestHive.{sparkContext, sql} +import org.apache.spark.sql.hive.test.TestHive.implicits._ case class Nested(a: Int, B: Int) case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested]) 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 dd0df1a9f6320..1e99003d3e9b5 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 @@ -47,7 +47,9 @@ case class ListStringCaseClass(l: Seq[String]) * A test suite for Hive custom UDFs. */ class HiveUdfSuite extends QueryTest { - import TestHive._ + + import TestHive.{udf, sql} + import TestHive.implicits._ test("spark sql udf test that returns a struct") { udf.register("getStruct", (_: Int) => Fields(1, 2, 3, 4, 5)) 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 268e5f6f01dff..22310ffadd25e 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 @@ -35,6 +35,9 @@ case class Nested3(f3: Int) * valid, but Hive currently cannot execute it. */ class SQLQuerySuite extends QueryTest { + + import org.apache.spark.sql.hive.test.TestHive.implicits._ + 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"), 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 79fd99d9f89ff..30441bbbdf817 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 @@ -149,6 +149,8 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll var partitionedTableDir: File = null var partitionedTableDirWithKey: File = null + import org.apache.spark.sql.hive.test.TestHive.implicits._ + override def beforeAll(): Unit = { partitionedTableDir = File.createTempFile("parquettests", "sparksql") partitionedTableDir.delete() From c3ba4d4cd032e376bfdf7ea7eaab65a79a771e7e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 4 Feb 2015 23:57:53 -0800 Subject: [PATCH 088/578] [MLlib] Minor: UDF style update. Author: Reynold Xin Closes #4388 from rxin/mllib-style and squashes the following commits: 61d465b [Reynold Xin] oops 3364295 [Reynold Xin] Missed one .. 5e068e3 [Reynold Xin] [MLlib] Minor: UDF style update. --- .../spark/ml/classification/LogisticRegression.scala | 8 +++++--- .../scala/org/apache/spark/ml/recommendation/ALS.scala | 4 ++-- 2 files changed, 7 insertions(+), 5 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 df90078de148f..b46a5cd8bdf29 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 @@ -132,12 +132,14 @@ class LogisticRegressionModel private[ml] ( override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { transformSchema(dataset.schema, paramMap, logging = true) val map = this.paramMap ++ paramMap - val scoreFunction = udf((v: Vector) => { + val scoreFunction = udf { v: Vector => val margin = BLAS.dot(v, weights) 1.0 / (1.0 + math.exp(-margin)) - } : Double) + } val t = map(threshold) - val predictFunction = udf((score: Double) => { if (score > t) 1.0 else 0.0 } : Double) + val predictFunction = udf { score: Double => + if (score > t) 1.0 else 0.0 + } dataset .select($"*", scoreFunction(col(map(featuresCol))).as(map(scoreCol))) .select($"*", predictFunction(col(map(scoreCol))).as(map(predictionCol))) 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 09456b289e2dc..bf5737177ceee 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 @@ -129,13 +129,13 @@ class ALSModel private[ml] ( // Register a UDF for DataFrame, and then // create a new column named map(predictionCol) by running the predict UDF. - val predict = udf((userFeatures: Seq[Float], itemFeatures: Seq[Float]) => { + val predict = udf { (userFeatures: Seq[Float], itemFeatures: Seq[Float]) => if (userFeatures != null && itemFeatures != null) { blas.sdot(k, userFeatures.toArray, 1, itemFeatures.toArray, 1) } else { Float.NaN } - } : Float) + } dataset .join(users, dataset(map(userCol)) === users("id"), "left") .join(items, dataset(map(itemCol)) === items("id"), "left") From 6580929fa029c4010dd4170de9be9f18516f8e5a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 5 Feb 2015 00:42:50 -0800 Subject: [PATCH 089/578] [HOTFIX] MLlib build break. --- .../org/apache/spark/mllib/classification/NaiveBayes.scala | 2 +- .../mllib/classification/impl/GLMClassificationModel.scala | 2 +- .../apache/spark/mllib/regression/impl/GLMRegressionModel.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index 4bafd495f90b1..d9ce2822dd391 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 @@ -92,7 +92,7 @@ object NaiveBayesModel extends Loader[NaiveBayesModel] { def save(sc: SparkContext, path: String, data: Data): Unit = { val sqlContext = new SQLContext(sc) - import sqlContext._ + import sqlContext.implicits._ // Create JSON metadata. val metadataRDD = 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 b60c0cdd0ab7b..8d600572ed7f3 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 @@ -49,7 +49,7 @@ private[classification] object GLMClassificationModel { intercept: Double, threshold: Option[Double]): Unit = { val sqlContext = new SQLContext(sc) - import sqlContext._ + import sqlContext.implicits._ // Create JSON metadata. val metadataRDD = 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 00f25a8be9397..838100e949ec2 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 @@ -45,7 +45,7 @@ private[regression] object GLMRegressionModel { weights: Vector, intercept: Double): Unit = { val sqlContext = new SQLContext(sc) - import sqlContext._ + import sqlContext.implicits._ // Create JSON metadata. val metadataRDD = From c4b1108c3f9658adebbdf8508d325528c3206f16 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 5 Feb 2015 10:15:55 -0800 Subject: [PATCH 090/578] SPARK-4687. Add a recursive option to the addFile API This adds a recursive option to the addFile API to satisfy Hive's needs. It only allows specifying HDFS dirs that will be copied down on every executor. There are a couple outstanding questions. * Should we allow specifying local dirs as well? The best way to do this would probably be to archive them. The drawback is that it would require a fair bit of code that I don't know of any current use cases for. * The addFiles implementation has a caching component that I don't entirely understand. What events are we caching between? AFAICT it's users calling addFile on the same file in the same app at different times? Do we want/need to add something similar for addDirectory. * The addFiles implementation will check to see if an added file already exists and has the same contents. I imagine we want the same behavior, so planning to add this unless people think otherwise. I plan to add some tests if people are OK with the approach. Author: Sandy Ryza Closes #3670 from sryza/sandy-spark-4687 and squashes the following commits: f9fc77f [Sandy Ryza] Josh's comments 70cd24d [Sandy Ryza] Add another test 13da824 [Sandy Ryza] Revert executor changes 38bf94d [Sandy Ryza] Marcelo's comments ca83849 [Sandy Ryza] Add addFile test 1941be3 [Sandy Ryza] Fix test and avoid HTTP server in local mode 31f15a9 [Sandy Ryza] Use cache recursively and fix some compile errors 0239c3d [Sandy Ryza] Change addDirectory to addFile with recursive 46fe70a [Sandy Ryza] SPARK-4687. Add a addDirectory API --- .../scala/org/apache/spark/SparkContext.scala | 68 +++++++++++--- .../scala/org/apache/spark/util/Utils.scala | 88 +++++++++++++++---- .../org/apache/spark/SparkContextSuite.scala | 77 ++++++++++++++++ .../org/apache/spark/util/UtilsSuite.scala | 31 +++++++ 4 files changed, 237 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a7adddb6c83ec..24490fddc5c6a 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -25,29 +25,37 @@ import java.net.URI import java.util.{Arrays, Properties, UUID} import java.util.concurrent.atomic.AtomicInteger import java.util.UUID.randomUUID + import scala.collection.{Map, Set} import scala.collection.JavaConversions._ 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, FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable} -import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, SequenceFileInputFormat, TextInputFormat} +import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, + FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable} +import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, SequenceFileInputFormat, + TextInputFormat} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob} import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} + import org.apache.mesos.MesosNativeLibrary -import akka.actor.Props 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.input.{StreamInputFormat, PortableDataStream, WholeTextFileInputFormat, FixedLengthBinaryInputFormat} +import org.apache.spark.input.{StreamInputFormat, PortableDataStream, WholeTextFileInputFormat, + FixedLengthBinaryInputFormat} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkDeploySchedulerBackend, SimrSchedulerBackend} +import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, + SparkDeploySchedulerBackend, SimrSchedulerBackend} import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage._ @@ -1016,12 +1024,48 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * filesystems), or an HTTP, HTTPS or FTP URI. To access the file in Spark jobs, * use `SparkFiles.get(fileName)` to find its download location. */ - def addFile(path: String) { + def addFile(path: String): Unit = { + addFile(path, false) + } + + /** + * Add a file to be downloaded with this Spark job on every node. + * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported + * filesystems), or an HTTP, HTTPS or FTP URI. To access the file in Spark jobs, + * use `SparkFiles.get(fileName)` to find its download location. + * + * A directory can be given if the recursive option is set to true. Currently directories are only + * supported for Hadoop-supported filesystems. + */ + def addFile(path: String, recursive: Boolean): Unit = { val uri = new URI(path) - val key = uri.getScheme match { - case null | "file" => env.httpFileServer.addFile(new File(uri.getPath)) - case "local" => "file:" + uri.getPath - case _ => path + val schemeCorrectedPath = uri.getScheme match { + case null | "local" => "file:" + uri.getPath + case _ => path + } + + val hadoopPath = new Path(schemeCorrectedPath) + val scheme = new URI(schemeCorrectedPath).getScheme + if (!Array("http", "https", "ftp").contains(scheme)) { + val fs = hadoopPath.getFileSystem(hadoopConfiguration) + if (!fs.exists(hadoopPath)) { + throw new FileNotFoundException(s"Added file $hadoopPath does not exist.") + } + val isDir = fs.isDirectory(hadoopPath) + if (!isLocal && scheme == "file" && isDir) { + throw new SparkException(s"addFile does not support local directories when not running " + + "local mode.") + } + if (!recursive && isDir) { + throw new SparkException(s"Added file $hadoopPath is a directory and recursive is not " + + "turned on.") + } + } + + val key = if (!isLocal && scheme == "file") { + env.httpFileServer.addFile(new File(uri.getPath)) + } else { + schemeCorrectedPath } val timestamp = System.currentTimeMillis addedFiles(key) = timestamp @@ -1633,8 +1677,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli val schedulingMode = getSchedulingMode.toString val addedJarPaths = addedJars.keys.toSeq val addedFilePaths = addedFiles.keys.toSeq - val environmentDetails = - SparkEnv.environmentDetails(conf, schedulingMode, addedJarPaths, addedFilePaths) + val environmentDetails = SparkEnv.environmentDetails(conf, schedulingMode, addedJarPaths, + addedFilePaths) val environmentUpdate = SparkListenerEnvironmentUpdate(environmentDetails) listenerBus.post(environmentUpdate) } 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 3c8a0e40bf785..72d15e65bcde6 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -386,8 +386,10 @@ private[spark] object Utils extends Logging { } /** - * Download a file to target directory. Supports fetching the file in a variety of ways, - * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter. + * Download a file or directory to target directory. Supports fetching the file in a variety of + * ways, including HTTP, Hadoop-compatible filesystems, and files on a standard filesystem, based + * on the URL parameter. Fetching directories is only supported from Hadoop-compatible + * filesystems. * * If `useCache` is true, first attempts to fetch the file to a local cache that's shared * across executors running the same application. `useCache` is used mainly for @@ -456,7 +458,6 @@ private[spark] object Utils extends Logging { * * @param url URL that `sourceFile` originated from, for logging purposes. * @param in InputStream to download. - * @param tempFile File path to download `in` to. * @param destFile File path to move `tempFile` to. * @param fileOverwrite Whether to delete/overwrite an existing `destFile` that does not match * `sourceFile` @@ -464,9 +465,11 @@ private[spark] object Utils extends Logging { private def downloadFile( url: String, in: InputStream, - tempFile: File, destFile: File, fileOverwrite: Boolean): Unit = { + val tempFile = File.createTempFile("fetchFileTemp", null, + new File(destFile.getParentFile.getAbsolutePath)) + logInfo(s"Fetching $url to $tempFile") try { val out = new FileOutputStream(tempFile) @@ -505,7 +508,7 @@ private[spark] object Utils extends Logging { removeSourceFile: Boolean = false): Unit = { if (destFile.exists) { - if (!Files.equal(sourceFile, destFile)) { + if (!filesEqualRecursive(sourceFile, destFile)) { if (fileOverwrite) { logInfo( s"File $destFile exists and does not match contents of $url, replacing it with $url" @@ -540,13 +543,44 @@ private[spark] object Utils extends Logging { Files.move(sourceFile, destFile) } else { logInfo(s"Copying ${sourceFile.getAbsolutePath} to ${destFile.getAbsolutePath}") - Files.copy(sourceFile, destFile) + copyRecursive(sourceFile, destFile) + } + } + + private def filesEqualRecursive(file1: File, file2: File): Boolean = { + if (file1.isDirectory && file2.isDirectory) { + val subfiles1 = file1.listFiles() + val subfiles2 = file2.listFiles() + if (subfiles1.size != subfiles2.size) { + return false + } + subfiles1.sortBy(_.getName).zip(subfiles2.sortBy(_.getName)).forall { + case (f1, f2) => filesEqualRecursive(f1, f2) + } + } else if (file1.isFile && file2.isFile) { + Files.equal(file1, file2) + } else { + false + } + } + + private def copyRecursive(source: File, dest: File): Unit = { + if (source.isDirectory) { + if (!dest.mkdir()) { + throw new IOException(s"Failed to create directory ${dest.getPath}") + } + val subfiles = source.listFiles() + subfiles.foreach(f => copyRecursive(f, new File(dest, f.getName))) + } else { + Files.copy(source, dest) } } /** - * Download a file to target directory. Supports fetching the file in a variety of ways, - * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter. + * Download a file or directory to target directory. Supports fetching the file in a variety of + * ways, including HTTP, Hadoop-compatible filesystems, and files on a standard filesystem, based + * on the URL parameter. Fetching directories is only supported from Hadoop-compatible + * filesystems. * * Throws SparkException if the target file already exists and has different contents than * the requested file. @@ -558,14 +592,11 @@ private[spark] object Utils extends Logging { conf: SparkConf, securityMgr: SecurityManager, hadoopConf: Configuration) { - val tempFile = File.createTempFile("fetchFileTemp", null, new File(targetDir.getAbsolutePath)) val targetFile = new File(targetDir, filename) val uri = new URI(url) val fileOverwrite = conf.getBoolean("spark.files.overwrite", defaultValue = false) Option(uri.getScheme).getOrElse("file") match { case "http" | "https" | "ftp" => - logInfo("Fetching " + url + " to " + tempFile) - var uc: URLConnection = null if (securityMgr.isAuthenticationEnabled()) { logDebug("fetchFile with security enabled") @@ -583,17 +614,44 @@ private[spark] object Utils extends Logging { uc.setReadTimeout(timeout) uc.connect() val in = uc.getInputStream() - downloadFile(url, in, tempFile, targetFile, fileOverwrite) + downloadFile(url, in, targetFile, fileOverwrite) case "file" => // In the case of a local file, copy the local file to the target directory. // Note the difference between uri vs url. val sourceFile = if (uri.isAbsolute) new File(uri) else new File(url) copyFile(url, sourceFile, targetFile, fileOverwrite) case _ => - // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others val fs = getHadoopFileSystem(uri, hadoopConf) - val in = fs.open(new Path(uri)) - downloadFile(url, in, tempFile, targetFile, fileOverwrite) + val path = new Path(uri) + fetchHcfsFile(path, new File(targetDir, path.getName), fs, conf, hadoopConf, fileOverwrite) + } + } + + /** + * Fetch a file or directory from a Hadoop-compatible filesystem. + * + * Visible for testing + */ + private[spark] def fetchHcfsFile( + path: Path, + targetDir: File, + fs: FileSystem, + conf: SparkConf, + hadoopConf: Configuration, + fileOverwrite: Boolean): Unit = { + if (!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) + } } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 8b3c6871a7b39..50f347f1954de 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -17,10 +17,17 @@ package org.apache.spark +import java.io.File + +import com.google.common.base.Charsets._ +import com.google.common.io.Files + import org.scalatest.FunSuite import org.apache.hadoop.io.BytesWritable +import org.apache.spark.util.Utils + class SparkContextSuite extends FunSuite with LocalSparkContext { test("Only one SparkContext may be active at a time") { @@ -72,4 +79,74 @@ 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 + try { + Files.write("somewords", file, UTF_8) + val length = file.length() + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addFile(file.getAbsolutePath) + 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") + } + if (length != gotten.length()) { + throw new SparkException( + s"file has different length $length than added file ${gotten.length()}") + } + if (absolutePath == gotten.getAbsolutePath) { + throw new SparkException("file should have been copied") + } + x + }).count() + } finally { + sc.stop() + } + } + + test("addFile recursive works") { + val pluto = Utils.createTempDir() + val neptune = Utils.createTempDir(pluto.getAbsolutePath) + val saturn = Utils.createTempDir(neptune.getAbsolutePath) + val alien1 = File.createTempFile("alien", "1", neptune) + val alien2 = File.createTempFile("alien", "2", saturn) + + try { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addFile(neptune.getAbsolutePath, true) + sc.parallelize(Array(1), 1).map(x => { + val sep = File.separator + if (!new File(SparkFiles.get(neptune.getName + sep + alien1.getName)).exists()) { + throw new SparkException("can't access file under root added directory") + } + if (!new File(SparkFiles.get(neptune.getName + sep + saturn.getName + sep + alien2.getName)) + .exists()) { + throw new SparkException("can't access file in nested directory") + } + if (new File(SparkFiles.get(pluto.getName + sep + neptune.getName + sep + alien1.getName)) + .exists()) { + throw new SparkException("file exists that shouldn't") + } + x + }).count() + } finally { + sc.stop() + } + } + + test("addFile recursive can't add directories by default") { + val dir = Utils.createTempDir() + + try { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + intercept[SparkException] { + sc.addFile(dir.getAbsolutePath) + } + } finally { + sc.stop() + } + } } 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 4544382094f96..fe2b644251157 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -29,6 +29,9 @@ import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.scalatest.FunSuite +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + import org.apache.spark.SparkConf class UtilsSuite extends FunSuite with ResetSystemProperties { @@ -381,4 +384,32 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { require(cnt === 2, "prepare should be called twice") require(time < 500, "preparation time should not count") } + + 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) + } + } } From 4d74f0601a2465b0d2273a8bcc716b304584831f Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 5 Feb 2015 11:12:50 -0800 Subject: [PATCH 091/578] [SPARK-5608] Improve SEO of Spark documentation pages - Add meta description tags on some of the most important doc pages - Shorten the titles of some pages to have more relevant keywords; for example there's no reason to have "Spark SQL Programming Guide - Spark 1.2.0 documentation", we can just say "Spark SQL - Spark 1.2.0 documentation". Author: Matei Zaharia Closes #4381 from mateiz/docs-seo and squashes the following commits: 4940563 [Matei Zaharia] [SPARK-5608] Improve SEO of Spark documentation pages --- docs/_layouts/global.html | 4 +++- docs/bagel-programming-guide.md | 3 ++- docs/configuration.md | 3 ++- docs/graphx-programming-guide.md | 4 +++- docs/index.md | 4 +++- docs/mllib-decision-tree.md | 4 ++-- docs/mllib-guide.md | 4 +++- docs/monitoring.md | 1 + docs/programming-guide.md | 1 + docs/quick-start.md | 1 + docs/security.md | 3 ++- docs/sql-programming-guide.md | 3 ++- docs/streaming-programming-guide.md | 4 +++- docs/tuning.md | 4 +++- 14 files changed, 31 insertions(+), 12 deletions(-) diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 8841f7675d35e..efc4c612937df 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -7,7 +7,9 @@ {{ page.title }} - Spark {{site.SPARK_VERSION_SHORT}} Documentation - + {% if page.description %} + + {% endif %} {% if page.redirect %} diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md index 7e55131754a3f..c2fe6b0e286ce 100644 --- a/docs/bagel-programming-guide.md +++ b/docs/bagel-programming-guide.md @@ -1,6 +1,7 @@ --- layout: global -title: Bagel Programming Guide +displayTitle: Bagel Programming Guide +title: Bagel --- **Bagel will soon be superseded by [GraphX](graphx-programming-guide.html); we recommend that new users try GraphX instead.** diff --git a/docs/configuration.md b/docs/configuration.md index 8b1d7598c47e4..4c86cb7c16238 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1,6 +1,7 @@ --- layout: global -title: Spark Configuration +displayTitle: Spark Configuration +title: Configuration --- * This will become a table of contents (this text will be scraped). {:toc} diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index e298c51f8a5b7..826f6d8f371c7 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -1,6 +1,8 @@ --- layout: global -title: GraphX Programming Guide +displayTitle: GraphX Programming Guide +title: GraphX +description: GraphX graph processing library guide for Spark SPARK_VERSION_SHORT --- * This will become a table of contents (this text will be scraped). diff --git a/docs/index.md b/docs/index.md index 171d6ddad62f3..e006be640e582 100644 --- a/docs/index.md +++ b/docs/index.md @@ -1,6 +1,8 @@ --- layout: global -title: Spark Overview +displayTitle: Spark Overview +title: Overview +description: Apache Spark SPARK_VERSION_SHORT documentation homepage --- Apache Spark is a fast and general-purpose cluster computing system. diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index fc8e732251a30..d1537def851e7 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -1,7 +1,7 @@ --- layout: global -title: Decision Tree - MLlib -displayTitle: MLlib - Decision Tree +title: Decision Trees - MLlib +displayTitle: MLlib - Decision Trees --- * Table of contents diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 39c64d06926bf..73728bb35eb96 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -1,6 +1,8 @@ --- layout: global -title: Machine Learning Library (MLlib) Programming Guide +title: MLlib +displayTitle: Machine Learning Library (MLlib) Guide +description: MLlib machine learning library overview for Spark SPARK_VERSION_SHORT --- MLlib is Spark's scalable machine learning library consisting of common learning algorithms and utilities, diff --git a/docs/monitoring.md b/docs/monitoring.md index f32cdef240d31..7a5cadc171d6d 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -1,6 +1,7 @@ --- layout: global title: Monitoring and Instrumentation +description: Monitoring, metrics, and instrumentation guide for Spark SPARK_VERSION_SHORT --- There are several ways to monitor Spark applications: web UIs, metrics, and external instrumentation. diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 6486614e71354..6b365e83fb56d 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -1,6 +1,7 @@ --- layout: global title: Spark Programming Guide +description: Spark SPARK_VERSION_SHORT programming guide in Java, Scala and Python --- * This will become a table of contents (this text will be scraped). diff --git a/docs/quick-start.md b/docs/quick-start.md index bf643bb70e153..81143da865cf0 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -1,6 +1,7 @@ --- layout: global title: Quick Start +description: Quick start tutorial for Spark SPARK_VERSION_SHORT --- * This will become a table of contents (this text will be scraped). diff --git a/docs/security.md b/docs/security.md index 6e0a54fbc4ad7..c034ba12ff1fc 100644 --- a/docs/security.md +++ b/docs/security.md @@ -1,6 +1,7 @@ --- layout: global -title: Spark Security +displayTitle: Spark Security +title: Security --- Spark currently supports authentication via a shared secret. Authentication can be configured to be on via the `spark.authenticate` configuration parameter. This parameter controls whether the Spark communication protocols do authentication using the shared secret. This authentication is a basic handshake to make sure both sides have the same shared secret and are allowed to communicate. If the shared secret is not identical they will not be allowed to communicate. The shared secret is created as follows: diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 22664b419f5cb..350df9a731b0a 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1,6 +1,7 @@ --- layout: global -title: Spark SQL Programming Guide +displayTitle: Spark SQL Programming Guide +title: Spark SQL --- * This will become a table of contents (this text will be scraped). diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index e37a2bb37b9a4..96fb12ce5e0b9 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -1,6 +1,8 @@ --- layout: global -title: Spark Streaming Programming Guide +displayTitle: Spark Streaming Programming Guide +title: Spark Streaming +description: Spark Streaming programming guide and tutorial for Spark SPARK_VERSION_SHORT --- * This will become a table of contents (this text will be scraped). diff --git a/docs/tuning.md b/docs/tuning.md index efaac9d3d405f..cbd227868b248 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -1,6 +1,8 @@ --- layout: global -title: Tuning Spark +displayTitle: Tuning Spark +title: Tuning +description: Tuning and performance optimization guide for Spark SPARK_VERSION_SHORT --- * This will become a table of contents (this text will be scraped). From 34147549a7ad188e5eae8d818d36ca0fe882c16f Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Thu, 5 Feb 2015 12:03:13 -0800 Subject: [PATCH 092/578] [SPARK-5474][Build]curl should support URL redirection in build/mvn Author: GuoQiang Li Closes #4263 from witgo/SPARK-5474 and squashes the following commits: ef397ff [GuoQiang Li] review commits a398324 [GuoQiang Li] curl should support URL redirection in build/mvn --- build/mvn | 10 +++++----- dev/check-license | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/build/mvn b/build/mvn index a87c5a26230c8..53babf54debb6 100755 --- a/build/mvn +++ b/build/mvn @@ -34,14 +34,14 @@ install_app() { local binary="${_DIR}/$3" # setup `curl` and `wget` silent options if we're running on Jenkins - local curl_opts="" + local curl_opts="-L" local wget_opts="" if [ -n "$AMPLAB_JENKINS" ]; then - curl_opts="-s" - wget_opts="--quiet" + curl_opts="-s ${curl_opts}" + wget_opts="--quiet ${wget_opts}" else - curl_opts="--progress-bar" - wget_opts="--progress=bar:force" + curl_opts="--progress-bar ${curl_opts}" + wget_opts="--progress=bar:force ${wget_opts}" fi if [ -z "$3" -o ! -f "$binary" ]; then diff --git a/dev/check-license b/dev/check-license index a006f65710d6d..39943f882b6ca 100755 --- a/dev/check-license +++ b/dev/check-license @@ -31,7 +31,7 @@ acquire_rat_jar () { printf "Attempting to fetch rat\n" JAR_DL="${JAR}.part" if [ $(command -v curl) ]; then - curl --silent "${URL}" > "$JAR_DL" && mv "$JAR_DL" "$JAR" + 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 From 081ac69f3dfc095f5e01248536f3ed2b83d5ebe2 Mon Sep 17 00:00:00 2001 From: Jacek Lewandowski Date: Thu, 5 Feb 2015 12:00:04 -0800 Subject: [PATCH 093/578] SPARK-5548: Fixed a race condition in AkkaUtilsSuite `Await.result` and `selection.resolveOne` runs the same timeout simultaneously. When `Await.result` timeout is reached first, then `TimeoutException` is thrown. On the other hand, when `selection.resolveOne` timeout is reached first, `ActorNotFoundException` is thrown. This is an obvious race condition and the easiest way to fix it is to increase the timeout of one method to make sure the code fails on the other method first. Author: Jacek Lewandowski Closes #4343 from jacek-lewandowski/SPARK-5548-1.3 and squashes the following commits: b9ba47e [Jacek Lewandowski] SPARK-5548: Fixed a race condition in AkkaUtilsSuite --- core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 39e5d367d676c..2cc5817758cf7 100644 --- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -371,7 +371,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemPro AkkaUtils.address(AkkaUtils.protocol(slaveSystem), "spark", "localhost", boundPort, "MapOutputTracker")) val timeout = AkkaUtils.lookupTimeout(conf) intercept[TimeoutException] { - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout * 2), timeout) } actorSystem.shutdown() From 6fa4ac1b007a545201d82603f09b0573f529a4e6 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Thu, 5 Feb 2015 12:42:27 -0800 Subject: [PATCH 094/578] [Branch-1.3] [DOC] doc fix for date Trivial fix. Author: Daoyuan Wang Closes #4400 from adrian-wang/docdate and squashes the following commits: 31bbe40 [Daoyuan Wang] doc fix for date --- 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 350df9a731b0a..38f617d0c836c 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1108,7 +1108,7 @@ in Hive deployments. 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. -* `UNION` type and `DATE` type +* `UNION` type * Unique join * Single query multi insert * Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at From a83936e109087b5cae8b9734032f2f331fdad2e3 Mon Sep 17 00:00:00 2001 From: wangfei Date: Thu, 5 Feb 2015 12:44:12 -0800 Subject: [PATCH 095/578] [SPARK-5617][SQL] fix test failure of SQLQuerySuite SQLQuerySuite test failure: [info] - simple select (22 milliseconds) [info] - sorting (722 milliseconds) [info] - external sorting (728 milliseconds) [info] - limit (95 milliseconds) [info] - date row *** FAILED *** (35 milliseconds) [info] Results do not match for query: [info] 'Limit 1 [info] 'Project [CAST(2015-01-28, DateType) AS c0#3630] [info] 'UnresolvedRelation [testData], None [info] [info] == Analyzed Plan == [info] Limit 1 [info] Project [CAST(2015-01-28, DateType) AS c0#3630] [info] LogicalRDD [key#0,value#1], MapPartitionsRDD[1] at mapPartitions at ExistingRDD.scala:35 [info] [info] == Physical Plan == [info] Limit 1 [info] Project [16463 AS c0#3630] [info] PhysicalRDD [key#0,value#1], MapPartitionsRDD[1] at mapPartitions at ExistingRDD.scala:35 [info] [info] == Results == [info] !== Correct Answer - 1 == == Spark Answer - 1 == [info] ![2015-01-28] [2015-01-27] (QueryTest.scala:77) [info] org.scalatest.exceptions.TestFailedException: [info] at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:495) [info] at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555) [info] at org.scalatest.Assertions$class.fail(Assertions.scala:1328) [info] at org.scalatest.FunSuite.fail(FunSuite.scala:1555) [info] at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:77) [info] at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:95) [info] at org.apache.spark.sql.SQLQuerySuite$$anonfun$23.apply$mcV$sp(SQLQuerySuite.scala:300) [info] at org.apache.spark.sql.SQLQuerySuite$$anonfun$23.apply(SQLQuerySuite.scala:300) [info] at org.apache.spark.sql.SQLQuerySuite$$anonfun$23.apply(SQLQuerySuite.scala:300) [info] at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22) [info] at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85) [info] at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) [info] at org.scalatest.Transformer.apply(Transformer.scala:22) [info] at org.scalatest.Transformer.apply(Transformer.scala:20) [info] at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166) [info] at org.scalatest.Suite$class.withFixture(Suite.scala:1122) [info] at org.scalatest.FunSuite.withFixture(FunSuite.scala:1555) [info] at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163) [info] at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) [info] at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) [info] at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306) [info] at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175) [info] at org.scalatest.FunSuite.runTest(FunSuite.scala:1555) [info] at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) [info] at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) [info] at org.scalatest.SuperEngine$$anonfun$traverseSubNode Author: wangfei Closes #4395 from scwf/SQLQuerySuite and squashes the following commits: 1431a2d [wangfei] fix conflicts c35fe5e [wangfei] minor fix 01dab3a [wangfei] fix test failure of SQLQuerySuite --- .../test/scala/org/apache/spark/sql/QueryTest.scala | 7 +++++++ .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 12 ------------ 2 files changed, 7 insertions(+), 12 deletions(-) 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 a7f2faa3ecf75..f9ddd2ca5c567 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 @@ -17,12 +17,19 @@ package org.apache.spark.sql +import java.util.{Locale, TimeZone} + import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.columnar.InMemoryRelation class QueryTest extends PlanTest { + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + // Add Locale setting + Locale.setDefault(Locale.US) + /** * Runs the plan and makes sure the answer contains all of the keywords, or the * none of keywords are listed in the answer 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 dc8ee41712fcd..11502edf972e9 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,8 +17,6 @@ package org.apache.spark.sql -import java.util.TimeZone - import org.apache.spark.sql.test.TestSQLContext import org.scalatest.BeforeAndAfterAll @@ -37,16 +35,6 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { import org.apache.spark.sql.test.TestSQLContext.implicits._ - var origZone: TimeZone = _ - override protected def beforeAll() { - origZone = TimeZone.getDefault - TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) - } - - override protected def afterAll() { - TimeZone.setDefault(origZone) - } - test("SPARK-4625 support SORT BY in SimpleSQLParser & DSL") { checkAnswer( sql("SELECT a FROM testData2 SORT BY a"), From 4d8d070c4f9f8211afb95d29036eb5e41796dcf2 Mon Sep 17 00:00:00 2001 From: OopsOutOfMemory Date: Thu, 5 Feb 2015 13:07:48 -0800 Subject: [PATCH 096/578] [SPARK-5135][SQL] Add support for describe table to DDL in SQLContext Hi, rxin marmbrus I considered your suggestion (in #4127) and now re-write it. This is now up-to-date. Could u please review it ? Author: OopsOutOfMemory Closes #4227 from OopsOutOfMemory/describe and squashes the following commits: 053826f [OopsOutOfMemory] describe --- .../apache/spark/sql/types/dataTypes.scala | 26 ++++- .../spark/sql/execution/SparkStrategies.scala | 13 ++- .../apache/spark/sql/execution/commands.scala | 10 +- .../org/apache/spark/sql/sources/ddl.scala | 39 +++++++- .../spark/sql/sources/DDLTestSuite.scala | 99 +++++++++++++++++++ .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../org/apache/spark/sql/hive/HiveQl.scala | 17 +--- .../spark/sql/hive/HiveStrategies.scala | 6 +- .../hive/execution/HiveComparisonTest.scala | 2 +- .../sql/hive/execution/HiveQuerySuite.scala | 4 +- 10 files changed, 190 insertions(+), 28 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala 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 a6d6ddd905393..be362be55b563 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 @@ -227,8 +227,9 @@ abstract class DataType { def json: String = compact(render(jsonValue)) def prettyJson: String = pretty(render(jsonValue)) -} + def simpleString: String = typeName +} /** * :: DeveloperApi :: @@ -242,7 +243,6 @@ case object NullType extends DataType { override def defaultSize: Int = 1 } - protected[sql] object NativeType { val all = Seq( IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) @@ -448,6 +448,8 @@ case object LongType extends IntegralType { * The default size of a value of the LongType is 8 bytes. */ override def defaultSize: Int = 8 + + override def simpleString = "bigint" } @@ -470,6 +472,8 @@ case object IntegerType extends IntegralType { * The default size of a value of the IntegerType is 4 bytes. */ override def defaultSize: Int = 4 + + override def simpleString = "int" } @@ -492,6 +496,8 @@ case object ShortType extends IntegralType { * The default size of a value of the ShortType is 2 bytes. */ override def defaultSize: Int = 2 + + override def simpleString = "smallint" } @@ -514,6 +520,8 @@ case object ByteType extends IntegralType { * The default size of a value of the ByteType is 1 byte. */ override def defaultSize: Int = 1 + + override def simpleString = "tinyint" } @@ -573,6 +581,11 @@ case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalT * The default size of a value of the DecimalType is 4096 bytes. */ override def defaultSize: Int = 4096 + + override def simpleString = precisionInfo match { + case Some(PrecisionInfo(precision, scale)) => s"decimal($precision,$scale)" + case None => "decimal(10,0)" + } } @@ -695,6 +708,8 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT * (We assume that there are 100 elements). */ override def defaultSize: Int = 100 * elementType.defaultSize + + override def simpleString = s"array<${elementType.simpleString}>" } @@ -870,6 +885,11 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru * The default size of a value of the StructType is the total default sizes of all field types. */ override def defaultSize: Int = fields.map(_.dataType.defaultSize).sum + + override def simpleString = { + val fieldTypes = fields.map(field => s"${field.name}:${field.dataType.simpleString}") + s"struct<${fieldTypes.mkString(",")}>" + } } @@ -920,6 +940,8 @@ case class MapType( * (We assume that there are 100 elements). */ override def defaultSize: Int = 100 * (keyType.defaultSize + valueType.defaultSize) + + override def simpleString = s"map<${keyType.simpleString},${valueType.simpleString}>" } 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 0c77d399b2eb8..f06f5fd1fc511 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 @@ -26,9 +26,10 @@ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} import org.apache.spark.sql.parquet._ import org.apache.spark.sql.types._ +import org.apache.spark.sql.sources.{DescribeCommand => LogicalDescribeCommand} +import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand} import org.apache.spark.sql.sources._ - private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { self: SQLContext#SparkPlanner => @@ -337,6 +338,16 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { 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 + 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 } } 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 1bc53968c4ca3..335757087deef 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 @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{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 @@ -176,9 +177,14 @@ case class UncacheTableCommand(tableName: String) extends RunnableCommand { @DeveloperApi case class DescribeCommand( child: SparkPlan, - override val output: Seq[Attribute]) extends RunnableCommand { + override val output: Seq[Attribute], + isExtended: Boolean) extends RunnableCommand { override def run(sqlContext: SQLContext) = { - child.output.map(field => Row(field.name, field.dataType.toString, null)) + child.schema.fields.map { field => + val cmtKey = "comment" + val comment = if (field.metadata.contains(cmtKey)) field.metadata.getString(cmtKey) else "" + Row(field.name, field.dataType.simpleString, comment) + } } } 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 ead827728cf4b..2ef740b3be0bd 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 @@ -23,6 +23,8 @@ import org.apache.spark.Logging import org.apache.spark.sql.{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.execution.RunnableCommand import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -50,7 +52,6 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { } } - // 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") @@ -61,6 +62,8 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { protected val EXISTS = Keyword("EXISTS") protected val USING = Keyword("USING") protected val OPTIONS = Keyword("OPTIONS") + protected val DESCRIBE = Keyword("DESCRIBE") + protected val EXTENDED = Keyword("EXTENDED") protected val AS = Keyword("AS") protected val COMMENT = Keyword("COMMENT") @@ -82,7 +85,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 + protected lazy val ddl: Parser[LogicalPlan] = createTable | describeTable protected def start: Parser[LogicalPlan] = ddl @@ -136,6 +139,22 @@ private[sql] class DDLParser extends AbstractSparkSQLParser with Logging { protected lazy val tableCols: Parser[Seq[StructField]] = "(" ~> repsep(column, ",") <~ ")" + /* + * describe [extended] table avroTable + * This will display all columns of table `avroTable` includes column_name,column_type,nullable + */ + protected lazy val describeTable: Parser[LogicalPlan] = + (DESCRIBE ~> opt(EXTENDED)) ~ (ident <~ ".").? ~ ident ^^ { + case e ~ db ~ tbl => + val tblIdentifier = db match { + case Some(dbName) => + Seq(dbName, tbl) + case None => + Seq(tbl) + } + DescribeCommand(UnresolvedRelation(tblIdentifier, None), e.isDefined) + } + protected lazy val options: Parser[Map[String, String]] = "(" ~> repsep(pair, ",") <~ ")" ^^ { case s: Seq[(String, String)] => s.toMap } @@ -274,6 +293,22 @@ object ResolvedDataSource { private[sql] case class ResolvedDataSource(provider: Class[_], relation: BaseRelation) +/** + * Returned for the "DESCRIBE [EXTENDED] [dbName.]tableName" command. + * @param table The table to be described. + * @param isExtended True if "DESCRIBE EXTENDED" is used. Otherwise, false. + * It is effective only when the table is a Hive table. + */ +private[sql] case class DescribeCommand( + table: LogicalPlan, + isExtended: Boolean) extends Command { + override def output = Seq( + // Column names are based on Hive. + AttributeReference("col_name", StringType, nullable = false)(), + AttributeReference("data_type", StringType, nullable = false)(), + AttributeReference("comment", StringType, nullable = false)()) +} + private[sql] case class CreateTableUsing( tableName: String, userSpecifiedSchema: Option[StructType], 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 new file mode 100644 index 0000000000000..0ec756bfeb7ef --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala @@ -0,0 +1,99 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF 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.apache.spark.sql._ +import org.apache.spark.sql.types._ + +class DDLScanSource extends RelationProvider { + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + SimpleDDLScan(parameters("from").toInt, parameters("TO").toInt)(sqlContext) + } +} + +case class SimpleDDLScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) + extends TableScan { + + override def schema = + StructType(Seq( + StructField("intType", IntegerType, nullable = false, + new MetadataBuilder().putString("comment", "test comment").build()), + StructField("stringType", StringType, nullable = false), + StructField("dateType", DateType, nullable = false), + StructField("timestampType", TimestampType, nullable = false), + StructField("doubleType", DoubleType, nullable = false), + StructField("bigintType", LongType, nullable = false), + StructField("tinyintType", ByteType, nullable = false), + StructField("decimalType", DecimalType.Unlimited, nullable = false), + StructField("fixedDecimalType", DecimalType(5,1), nullable = false), + StructField("binaryType", BinaryType, nullable = false), + StructField("booleanType", BooleanType, nullable = false), + StructField("smallIntType", ShortType, nullable = false), + StructField("floatType", FloatType, nullable = false), + StructField("mapType", MapType(StringType, StringType)), + StructField("arrayType", ArrayType(StringType)), + StructField("structType", + StructType(StructField("f1",StringType) :: + (StructField("f2",IntegerType)) :: Nil + ) + ) + )) + + + override def buildScan() = sqlContext.sparkContext.parallelize(from to to). + map(e => Row(s"people$e", e * 2)) +} + +class DDLTestSuite extends DataSourceTest { + import caseInsensisitiveContext._ + + before { + sql( + """ + |CREATE TEMPORARY TABLE ddlPeople + |USING org.apache.spark.sql.sources.DDLScanSource + |OPTIONS ( + | From '1', + | To '10' + |) + """.stripMargin) + } + + sqlTest( + "describe ddlPeople", + Seq( + Row("intType", "int", "test comment"), + Row("stringType", "string", ""), + Row("dateType", "date", ""), + Row("timestampType", "timestamp", ""), + Row("doubleType", "double", ""), + Row("bigintType", "bigint", ""), + Row("tinyintType", "tinyint", ""), + Row("decimalType", "decimal(10,0)", ""), + Row("fixedDecimalType", "decimal(5,1)", ""), + Row("binaryType", "binary", ""), + Row("booleanType", "boolean", ""), + Row("smallIntType", "smallint", ""), + Row("floatType", "float", ""), + Row("mapType", "map", ""), + Row("arrayType", "array", ""), + Row("structType", "struct", "") + )) +} 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 1921bf6e5e1a6..d2371d4a5519e 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 @@ -75,7 +75,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { DataFrame(this, ddlParser(sqlText, exceptionOnError = false).getOrElse(HiveQl.parseSql(substituted))) } else { - sys.error(s"Unsupported SQL dialect: ${conf.dialect}. Try 'sql' or 'hiveql'") + sys.error(s"Unsupported SQL dialect: ${conf.dialect}. Try 'sql' or 'hiveql'") } } 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 62e9d92eac076..c19a091719190 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 @@ -34,6 +34,7 @@ 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.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._ @@ -47,22 +48,6 @@ import scala.collection.JavaConversions._ */ private[hive] case object NativePlaceholder extends Command -/** - * Returned for the "DESCRIBE [EXTENDED] [dbName.]tableName" command. - * @param table The table to be described. - * @param isExtended True if "DESCRIBE EXTENDED" is used. Otherwise, false. - * It is effective only when the table is a Hive table. - */ -case class DescribeCommand( - table: LogicalPlan, - isExtended: Boolean) extends Command { - override def output = Seq( - // Column names are based on Hive. - AttributeReference("col_name", StringType, nullable = false)(), - AttributeReference("data_type", StringType, nullable = false)(), - AttributeReference("comment", StringType, nullable = false)()) -} - /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ private[hive] object HiveQl { protected val nativeCommands = Seq( 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 d89111094b9ff..7857a0252ebb3 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 @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate 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.hive.execution._ @@ -240,8 +241,11 @@ private[hive] trait HiveStrategies { case t: MetastoreRelation => ExecutedCommand( DescribeHiveTableCommand(t, describe.output, describe.isExtended)) :: Nil + case o: LogicalPlan => - ExecutedCommand(RunnableDescribeCommand(planLater(o), describe.output)) :: Nil + val resultPlan = context.executePlan(o).executedPlan + ExecutedCommand(RunnableDescribeCommand( + resultPlan, describe.output, describe.isExtended)) :: Nil } case _ => Nil 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 f8a957d55d57e..a90bd1e257ade 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 @@ -22,8 +22,8 @@ import java.io._ import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} import org.apache.spark.Logging +import org.apache.spark.sql.sources.DescribeCommand import org.apache.spark.sql.execution.{SetCommand, ExplainCommand} -import org.apache.spark.sql.hive.DescribeCommand import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util._ 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 407d6058c33ed..bb73ff1ea7e43 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 @@ -741,8 +741,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { assertResult( Array( - Row("a", "IntegerType", null), - Row("b", "StringType", null)) + Row("a", "int", ""), + Row("b", "string", "")) ) { sql("DESCRIBE test_describe_commands2") .select('col_name, 'data_type, 'comment) From 62371adaa5b9251579db7300504506975689610c Mon Sep 17 00:00:00 2001 From: x1- Date: Thu, 5 Feb 2015 15:02:04 -0800 Subject: [PATCH 097/578] [SPARK-5460][MLlib] Wrapped `Try` around `deleteAllCheckpoints` - RandomForest. Because `deleteAllCheckpoints` has IOException potential. fix issue. Author: x1- Closes #4347 from x1-/SPARK-5460 and squashes the following commits: 7a3d8de [x1-] change `Try()` to `try catch { case ... }` ar RandomForest. 3a52745 [x1-] modified typo. 'faild' -> 'failed' and remove disused '-'. 1572576 [x1-] Wrapped `Try` around `deleteAllCheckpoints` - RandomForest. --- .../scala/org/apache/spark/mllib/tree/RandomForest.scala | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) 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 482dd4b272d1d..45b0154c5e4cb 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 @@ -17,6 +17,8 @@ package org.apache.spark.mllib.tree +import java.io.IOException + import scala.collection.mutable import scala.collection.JavaConverters._ @@ -244,7 +246,12 @@ private class RandomForest ( // Delete any remaining checkpoints used for node Id cache. if (nodeIdCache.nonEmpty) { - nodeIdCache.get.deleteAllCheckpoints() + try { + nodeIdCache.get.deleteAllCheckpoints() + } catch { + case e:IOException => + logWarning(s"delete all chackpoints failed. Error reason: ${e.getMessage}") + } } val trees = topNodes.map(topNode => new DecisionTreeModel(topNode, strategy.algo)) From c19152cd2a5d407ecf526a90e3bb059f09905b3a Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 5 Feb 2015 15:07:33 -0800 Subject: [PATCH 098/578] [SPARK-5604[MLLIB] remove checkpointDir from LDA `checkpointDir` is a Spark global configuration. Users should set it outside LDA. This PR also hides some methods under `private[clustering] object LDA`, so they don't show up in the generated Java doc (SPARK-5610). jkbradley Author: Xiangrui Meng Closes #4390 from mengxr/SPARK-5604 and squashes the following commits: a34bb39 [Xiangrui Meng] remove checkpointDir from LDA --- .../spark/examples/mllib/LDAExample.scala | 2 +- .../apache/spark/mllib/clustering/LDA.scala | 73 +++++-------------- .../impl/PeriodicGraphCheckpointer.scala | 8 -- .../impl/PeriodicGraphCheckpointerSuite.scala | 6 +- 4 files changed, 24 insertions(+), 65 deletions(-) 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 f4c545ad70e96..0e1b27a8bd2ee 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 @@ -134,7 +134,7 @@ object LDAExample { .setTopicConcentration(params.topicConcentration) .setCheckpointInterval(params.checkpointInterval) if (params.checkpointDir.nonEmpty) { - lda.setCheckpointDir(params.checkpointDir.get) + sc.setCheckpointDir(params.checkpointDir.get) } val startTime = System.nanoTime() val ldaModel = lda.run(corpus) 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 d8f82867a09d2..a1d3df03a1140 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 @@ -52,6 +52,9 @@ import org.apache.spark.util.Utils * - Paper which clearly explains several algorithms, including EM: * Asuncion, Welling, Smyth, and Teh. * "On Smoothing and Inference for Topic Models." UAI, 2009. + * + * @see [[http://en.wikipedia.org/wiki/Latent_Dirichlet_allocation Latent Dirichlet allocation + * (Wikipedia)]] */ @Experimental class LDA private ( @@ -60,11 +63,10 @@ class LDA private ( private var docConcentration: Double, private var topicConcentration: Double, private var seed: Long, - private var checkpointDir: Option[String], private var checkpointInterval: Int) extends Logging { def this() = this(k = 10, maxIterations = 20, docConcentration = -1, topicConcentration = -1, - seed = Utils.random.nextLong(), checkpointDir = None, checkpointInterval = 10) + seed = Utils.random.nextLong(), checkpointInterval = 10) /** * Number of topics to infer. I.e., the number of soft cluster centers. @@ -200,50 +202,18 @@ class LDA private ( this } - /** - * Directory for storing checkpoint files during learning. - * This is not necessary, but checkpointing helps with recovery (when nodes fail). - * It also helps with eliminating temporary shuffle files on disk, which can be important when - * LDA is run for many iterations. - */ - def getCheckpointDir: Option[String] = checkpointDir - - /** - * Directory for storing checkpoint files during learning. - * This is not necessary, but checkpointing helps with recovery (when nodes fail). - * It also helps with eliminating temporary shuffle files on disk, which can be important when - * LDA is run for many iterations. - * - * NOTE: If the [[org.apache.spark.SparkContext.checkpointDir]] is already set, then the value - * given to LDA is ignored, and the existing directory is kept. - * - * (default = None) - */ - def setCheckpointDir(checkpointDir: String): this.type = { - this.checkpointDir = Some(checkpointDir) - this - } - - /** - * Clear the directory for storing checkpoint files during learning. - * If one is already set in the [[org.apache.spark.SparkContext]], then checkpointing will still - * occur; otherwise, no checkpointing will be used. - */ - def clearCheckpointDir(): this.type = { - this.checkpointDir = None - this - } - /** * Period (in iterations) between checkpoints. - * @see [[getCheckpointDir]] */ def getCheckpointInterval: Int = checkpointInterval /** - * Period (in iterations) between checkpoints. - * (default = 10) - * @see [[getCheckpointDir]] + * Period (in iterations) between checkpoints (default = 10). Checkpointing helps with recovery + * (when nodes fail). It also helps with eliminating temporary shuffle files on disk, which can be + * important when LDA is run for many iterations. If the checkpoint directory is not set in + * [[org.apache.spark.SparkContext]], this setting is ignored. + * + * @see [[org.apache.spark.SparkContext#setCheckpointDir]] */ def setCheckpointInterval(checkpointInterval: Int): this.type = { this.checkpointInterval = checkpointInterval @@ -261,7 +231,7 @@ class LDA private ( */ def run(documents: RDD[(Long, Vector)]): DistributedLDAModel = { val state = LDA.initialState(documents, k, getDocConcentration, getTopicConcentration, seed, - checkpointDir, checkpointInterval) + checkpointInterval) var iter = 0 val iterationTimes = Array.fill[Double](maxIterations)(0) while (iter < maxIterations) { @@ -337,18 +307,18 @@ private[clustering] object LDA { * Vector over topics (length k) of token counts. * The meaning of these counts can vary, and it may or may not be normalized to be a distribution. */ - type TopicCounts = BDV[Double] + private[clustering] type TopicCounts = BDV[Double] - type TokenCount = Double + private[clustering] type TokenCount = Double /** Term vertex IDs are {-1, -2, ..., -vocabSize} */ - def term2index(term: Int): Long = -(1 + term.toLong) + private[clustering] def term2index(term: Int): Long = -(1 + term.toLong) - def index2term(termIndex: Long): Int = -(1 + termIndex).toInt + private[clustering] def index2term(termIndex: Long): Int = -(1 + termIndex).toInt - def isDocumentVertex(v: (VertexId, _)): Boolean = v._1 >= 0 + private[clustering] def isDocumentVertex(v: (VertexId, _)): Boolean = v._1 >= 0 - def isTermVertex(v: (VertexId, _)): Boolean = v._1 < 0 + private[clustering] def isTermVertex(v: (VertexId, _)): Boolean = v._1 < 0 /** * Optimizer for EM algorithm which stores data + parameter graph, plus algorithm parameters. @@ -360,17 +330,16 @@ private[clustering] object LDA { * @param docConcentration "alpha" * @param topicConcentration "beta" or "eta" */ - class EMOptimizer( + private[clustering] class EMOptimizer( var graph: Graph[TopicCounts, TokenCount], val k: Int, val vocabSize: Int, val docConcentration: Double, val topicConcentration: Double, - checkpointDir: Option[String], checkpointInterval: Int) { private[LDA] val graphCheckpointer = new PeriodicGraphCheckpointer[TopicCounts, TokenCount]( - graph, checkpointDir, checkpointInterval) + graph, checkpointInterval) def next(): EMOptimizer = { val eta = topicConcentration @@ -468,7 +437,6 @@ private[clustering] object LDA { docConcentration: Double, topicConcentration: Double, randomSeed: Long, - checkpointDir: Option[String], checkpointInterval: Int): EMOptimizer = { // For each document, create an edge (Document -> Term) for each unique term in the document. val edges: RDD[Edge[TokenCount]] = docs.flatMap { case (docID: Long, termCounts: Vector) => @@ -512,8 +480,7 @@ private[clustering] object LDA { val graph = Graph(docVertices ++ termVertices, edges) .partitionBy(PartitionStrategy.EdgePartition1D) - new EMOptimizer(graph, k, vocabSize, docConcentration, topicConcentration, checkpointDir, - checkpointInterval) + new EMOptimizer(graph, k, vocabSize, docConcentration, topicConcentration, checkpointInterval) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala index 76672fe51e834..6e5dd119dd653 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointer.scala @@ -74,7 +74,6 @@ import org.apache.spark.storage.StorageLevel * }}} * * @param currentGraph Initial graph - * @param checkpointDir The directory for storing checkpoint files * @param checkpointInterval Graphs will be checkpointed at this interval * @tparam VD Vertex descriptor type * @tparam ED Edge descriptor type @@ -83,7 +82,6 @@ import org.apache.spark.storage.StorageLevel */ private[mllib] class PeriodicGraphCheckpointer[VD, ED]( var currentGraph: Graph[VD, ED], - val checkpointDir: Option[String], val checkpointInterval: Int) extends Logging { /** FIFO queue of past checkpointed RDDs */ @@ -101,12 +99,6 @@ private[mllib] class PeriodicGraphCheckpointer[VD, ED]( */ private val sc = currentGraph.vertices.sparkContext - // If a checkpoint directory is given, and there's no prior checkpoint directory, - // then set the checkpoint directory with the given one. - if (checkpointDir.nonEmpty && sc.getCheckpointDir.isEmpty) { - sc.setCheckpointDir(checkpointDir.get) - } - updateGraph(currentGraph) /** diff --git a/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala index dac28a369b5b2..699f009f0f2ec 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/impl/PeriodicGraphCheckpointerSuite.scala @@ -38,7 +38,7 @@ class PeriodicGraphCheckpointerSuite extends FunSuite with MLlibTestSparkContext var graphsToCheck = Seq.empty[GraphToCheck] val graph1 = createGraph(sc) - val checkpointer = new PeriodicGraphCheckpointer(graph1, None, 10) + val checkpointer = new PeriodicGraphCheckpointer(graph1, 10) graphsToCheck = graphsToCheck :+ GraphToCheck(graph1, 1) checkPersistence(graphsToCheck, 1) @@ -57,9 +57,9 @@ class PeriodicGraphCheckpointerSuite extends FunSuite with MLlibTestSparkContext val path = tempDir.toURI.toString val checkpointInterval = 2 var graphsToCheck = Seq.empty[GraphToCheck] - + sc.setCheckpointDir(path) val graph1 = createGraph(sc) - val checkpointer = new PeriodicGraphCheckpointer(graph1, Some(path), checkpointInterval) + val checkpointer = new PeriodicGraphCheckpointer(graph1, checkpointInterval) graph1.edges.count() graph1.vertices.count() graphsToCheck = graphsToCheck :+ GraphToCheck(graph1, 1) From a9ed51178c89d83aae1ad420fb3f4a7f4d1812ec Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 5 Feb 2015 15:29:56 -0800 Subject: [PATCH 099/578] [SPARK-5182] [SPARK-5528] [SPARK-5509] [SPARK-3575] [SQL] Parquet data source improvements This PR adds three major improvements to Parquet data source: 1. Partition discovery While reading Parquet files resides in Hive style partition directories, `ParquetRelation2` automatically discovers partitioning information and infers partition column types. This is also a partial work for [SPARK-5182] [1], which aims to provide first class partitioning support for the data source API. Related code in this PR can be easily extracted to the data source API level in future versions. 1. Schema merging When enabled, Parquet data source collects schema information from all Parquet part-files and tries to merge them. Exceptions are thrown when incompatible schemas are detected. This feature is controlled by data source option `parquet.mergeSchema`, and is enabled by default. 1. Metastore Parquet table conversion moved to analysis phase This greatly simplifies the conversion logic. `ParquetConversion` strategy can be removed once the old Parquet implementation is removed in the future. This version of Parquet data source aims to entirely replace the old Parquet implementation. However, the old version hasn't been removed yet. Users can fall back to the old version by turning off SQL configuration `spark.sql.parquet.useDataSourceApi`. Other JIRA tickets fixed as side effects in this PR: - [SPARK-5509] [3]: `EqualTo` now uses a proper `Ordering` to compare binary types. - [SPARK-3575] [4]: Metastore schema is now preserved and passed to `ParquetRelation2` via data source option `parquet.metastoreSchema`. TODO: - [ ] More test cases for partition discovery - [x] Fix write path after data source write support (#4294) is merged It turned out to be non-trivial to fall back to old Parquet implementation on the write path when Parquet data source is enabled. Since we're planning to include data source write support in 1.3.0, I simply ignored two test cases involving Parquet insertion for now. - [ ] Fix outdated comments and documentations PS: This PR looks big, but more than a half of the changed lines in this PR are trivial changes to test cases. To test Parquet with and without the new data source, almost all Parquet test cases are moved into wrapper driver functions. This introduces hundreds of lines of changes. [1]: https://issues.apache.org/jira/browse/SPARK-5182 [2]: https://issues.apache.org/jira/browse/SPARK-5528 [3]: https://issues.apache.org/jira/browse/SPARK-5509 [4]: https://issues.apache.org/jira/browse/SPARK-3575 [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4308) Author: Cheng Lian Closes #4308 from liancheng/parquet-partition-discovery and squashes the following commits: b6946e6 [Cheng Lian] Fixes MiMA issues, addresses comments 8232e17 [Cheng Lian] Write support for Parquet data source a49bd28 [Cheng Lian] Fixes spelling typo in trait name "CreateableRelationProvider" 808380f [Cheng Lian] Fixes issues introduced while rebasing 50dd8d1 [Cheng Lian] Addresses @rxin's comment, fixes UDT schema merging adf2aae [Cheng Lian] Fixes compilation error introduced while rebasing 4e0175f [Cheng Lian] Fixes Python Parquet API, we need Py4J array to call varargs method 0d8ec1d [Cheng Lian] Adds more test cases b35c8c6 [Cheng Lian] Fixes some typos and outdated comments dd704fd [Cheng Lian] Fixes Python Parquet API 596c312 [Cheng Lian] Uses switch to control whether use Parquet data source or not 7d0f7a2 [Cheng Lian] Fixes Metastore Parquet table conversion a1896c7 [Cheng Lian] Fixes all existing Parquet test suites except for ParquetMetastoreSuite 5654c9d [Cheng Lian] Draft version of Parquet partition discovery and schema merging --- .../apache/spark/deploy/SparkHadoopUtil.scala | 17 +- python/pyspark/sql.py | 9 +- .../sql/catalyst/expressions/predicates.scala | 7 +- .../apache/spark/sql/types/dataTypes.scala | 68 +- .../org/apache/spark/sql/DataFrameImpl.scala | 6 +- .../scala/org/apache/spark/sql/SQLConf.scala | 5 + .../org/apache/spark/sql/SQLContext.scala | 20 +- .../spark/sql/execution/SparkStrategies.scala | 9 +- .../apache/spark/sql/json/JSONRelation.scala | 2 +- .../sql/parquet/ParquetTableSupport.scala | 9 +- .../spark/sql/parquet/ParquetTypes.scala | 17 +- .../apache/spark/sql/parquet/newParquet.scala | 753 ++++++++++++++---- .../sql/sources/DataSourceStrategy.scala | 8 +- .../org/apache/spark/sql/sources/ddl.scala | 20 +- .../apache/spark/sql/sources/interfaces.scala | 2 +- .../sql/parquet/ParquetFilterSuite.scala | 375 +++++---- .../spark/sql/parquet/ParquetIOSuite.scala | 353 ++++---- .../ParquetPartitionDiscoverySuite.scala | 126 +++ .../spark/sql/parquet/ParquetQuerySuite.scala | 127 +-- .../sql/parquet/ParquetSchemaSuite.scala | 37 + .../spark/sql/hive/HiveMetastoreCatalog.scala | 20 +- .../spark/sql/hive/HiveStrategies.scala | 7 +- .../spark/sql/parquet/HiveParquetSuite.scala | 78 +- .../spark/sql/parquet/parquetSuites.scala | 202 ++--- 24 files changed, 1541 insertions(+), 736 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.scala 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 d68854214ef06..03238e9fa0088 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -21,7 +21,7 @@ import java.lang.reflect.Method import java.security.PrivilegedExceptionAction import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} +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} @@ -191,6 +191,21 @@ class SparkHadoopUtil extends Logging { val method = context.getClass.getMethod("getConfiguration") method.invoke(context).asInstanceOf[Configuration] } + + /** + * Get [[FileStatus]] objects for all leaf children (files) under the given base path. If the + * given path points to a file, return a single-element collection containing [[FileStatus]] of + * that file. + */ + def listLeafStatuses(fs: FileSystem, basePath: Path): Seq[FileStatus] = { + def recurse(path: Path) = { + val (directories, leaves) = fs.listStatus(path).partition(_.isDir) + leaves ++ directories.flatMap(f => listLeafStatuses(fs, f.getPath)) + } + + val baseStatus = fs.getFileStatus(basePath) + if (baseStatus.isDir) recurse(basePath) else Array(baseStatus) + } } object SparkHadoopUtil { diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 3ac8ea597e142..e55f285a778c4 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1471,7 +1471,7 @@ def registerRDDAsTable(self, rdd, tableName): else: raise ValueError("Can only register DataFrame as table") - def parquetFile(self, path): + def parquetFile(self, *paths): """Loads a Parquet file, returning the result as a L{DataFrame}. >>> import tempfile, shutil @@ -1483,7 +1483,12 @@ def parquetFile(self, path): >>> sorted(df.collect()) == sorted(df2.collect()) True """ - jdf = self._ssql_ctx.parquetFile(path) + 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): 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 c84cc95520a19..365b1685a8e71 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.types.BooleanType +import org.apache.spark.sql.types.{BinaryType, BooleanType} object InterpretedPredicate { def apply(expression: Expression, inputSchema: Seq[Attribute]): (Row => Boolean) = @@ -175,7 +175,10 @@ case class EqualTo(left: Expression, right: Expression) extends BinaryComparison null } else { val r = right.eval(input) - if (r == null) null else l == r + if (r == null) null + else if (left.dataType != BinaryType) l == r + else BinaryType.ordering.compare( + l.asInstanceOf[Array[Byte]], r.asInstanceOf[Array[Byte]]) == 0 } } } 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 be362be55b563..91efe320546a7 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.types import java.sql.Timestamp +import scala.collection.mutable.ArrayBuffer import scala.math.Numeric.{FloatAsIfIntegral, DoubleAsIfIntegral} import scala.reflect.ClassTag import scala.reflect.runtime.universe.{TypeTag, runtimeMirror, typeTag} @@ -29,6 +30,7 @@ import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ +import org.apache.spark.SparkException import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.ScalaReflectionLock import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} @@ -159,7 +161,6 @@ object DataType { case failure: NoSuccess => throw new IllegalArgumentException(s"Unsupported dataType: $asString, $failure") } - } protected[types] def buildFormattedString( @@ -754,6 +755,57 @@ object StructType { def apply(fields: java.util.List[StructField]): StructType = { StructType(fields.toArray.asInstanceOf[Array[StructField]]) } + + private[sql] def merge(left: DataType, right: DataType): DataType = + (left, right) match { + case (ArrayType(leftElementType, leftContainsNull), + ArrayType(rightElementType, rightContainsNull)) => + ArrayType( + merge(leftElementType, rightElementType), + leftContainsNull || rightContainsNull) + + case (MapType(leftKeyType, leftValueType, leftContainsNull), + MapType(rightKeyType, rightValueType, rightContainsNull)) => + MapType( + merge(leftKeyType, rightKeyType), + merge(leftValueType, rightValueType), + leftContainsNull || rightContainsNull) + + case (StructType(leftFields), StructType(rightFields)) => + val newFields = ArrayBuffer.empty[StructField] + + leftFields.foreach { + case leftField @ StructField(leftName, leftType, leftNullable, _) => + rightFields + .find(_.name == leftName) + .map { case rightField @ StructField(_, rightType, rightNullable, _) => + leftField.copy( + dataType = merge(leftType, rightType), + nullable = leftNullable || rightNullable) + } + .orElse(Some(leftField)) + .foreach(newFields += _) + } + + rightFields + .filterNot(f => leftFields.map(_.name).contains(f.name)) + .foreach(newFields += _) + + StructType(newFields) + + case (DecimalType.Fixed(leftPrecision, leftScale), + DecimalType.Fixed(rightPrecision, rightScale)) => + DecimalType(leftPrecision.max(rightPrecision), leftScale.max(rightScale)) + + case (leftUdt: UserDefinedType[_], rightUdt: UserDefinedType[_]) + if leftUdt.userClass == rightUdt.userClass => leftUdt + + case (leftType, rightType) if leftType == rightType => + leftType + + case _ => + throw new SparkException(s"Failed to merge incompatible data types $left and $right") + } } @@ -890,6 +942,20 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru val fieldTypes = fields.map(field => s"${field.name}:${field.dataType.simpleString}") s"struct<${fieldTypes.mkString(",")}>" } + + /** + * Merges with another schema (`StructType`). For a struct field A from `this` and a struct field + * B from `that`, + * + * 1. If A and B have the same name and data type, they are merged to a field C with the same name + * and data type. C is nullable if and only if either A or B is nullable. + * 2. If A doesn't exist in `that`, it's included in the result schema. + * 3. If B doesn't exist in `this`, it's also included in the result schema. + * 4. Otherwise, `this` and `that` are considered as conflicting schemas and an exception would be + * thrown. + */ + private[sql] def merge(that: StructType): StructType = + StructType.merge(this, that).asInstanceOf[StructType] } 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 d6df927f9d42c..58d11751353b3 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 @@ -295,7 +295,11 @@ private[sql] class DataFrameImpl protected[sql]( } override def saveAsParquetFile(path: String): Unit = { - sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd + if (sqlContext.conf.parquetUseDataSourceApi) { + save("org.apache.spark.sql.parquet", "path" -> path) + } else { + sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd + } } override def saveAsTable(tableName: String): Unit = { 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 7fe17944a734e..5ef3bb022fc5a 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 @@ -37,6 +37,7 @@ private[spark] object SQLConf { val PARQUET_CACHE_METADATA = "spark.sql.parquet.cacheMetadata" val PARQUET_COMPRESSION = "spark.sql.parquet.compression.codec" val PARQUET_FILTER_PUSHDOWN_ENABLED = "spark.sql.parquet.filterPushdown" + val PARQUET_USE_DATA_SOURCE_API = "spark.sql.parquet.useDataSourceApi" val COLUMN_NAME_OF_CORRUPT_RECORD = "spark.sql.columnNameOfCorruptRecord" val BROADCAST_TIMEOUT = "spark.sql.broadcastTimeout" @@ -105,6 +106,10 @@ private[sql] class SQLConf extends Serializable { private[spark] def parquetFilterPushDown = getConf(PARQUET_FILTER_PUSHDOWN_ENABLED, "false").toBoolean + /** When true uses Parquet implementation based on data source API */ + private[spark] def parquetUseDataSourceApi = + getConf(PARQUET_USE_DATA_SOURCE_API, "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/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 01620aa0acd49..706ef6ad4f174 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 @@ -20,14 +20,13 @@ package org.apache.spark.sql import java.beans.Introspector import java.util.Properties -import scala.collection.immutable import scala.collection.JavaConversions._ +import scala.collection.immutable import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.{SparkContext, Partition} import org.apache.spark.annotation.{AlphaComponent, DeveloperApi, Experimental} -import org.apache.spark.api.java.{JavaSparkContext, JavaRDD} +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._ @@ -36,11 +35,12 @@ 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.rules.RuleExecutor import org.apache.spark.sql.execution._ -import org.apache.spark.sql.json._ import org.apache.spark.sql.jdbc.{JDBCPartition, JDBCPartitioningInfo, JDBCRelation} -import org.apache.spark.sql.sources._ +import org.apache.spark.sql.json._ +import org.apache.spark.sql.sources.{BaseRelation, DDLParser, DataSourceStrategy, LogicalRelation, _} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils +import org.apache.spark.{Partition, SparkContext} /** * :: AlphaComponent :: @@ -303,8 +303,14 @@ class SQLContext(@transient val sparkContext: SparkContext) * * @group userf */ - def parquetFile(path: String): DataFrame = - DataFrame(this, parquet.ParquetRelation(path, Some(sparkContext.hadoopConfiguration), this)) + @scala.annotation.varargs + def parquetFile(path: String, paths: String*): DataFrame = + if (conf.parquetUseDataSourceApi) { + baseRelationToDataFrame(parquet.ParquetRelation2(path +: paths, Map.empty)(this)) + } else { + DataFrame(this, parquet.ParquetRelation( + paths.mkString(","), Some(sparkContext.hadoopConfiguration), this)) + } /** * Loads a JSON file (one object per line), returning the result as a [[DataFrame]]. 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 f06f5fd1fc511..81bcf5a6f32dd 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 @@ -17,18 +17,17 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.{SQLContext, Strategy, execution} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} +import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand} import org.apache.spark.sql.parquet._ +import org.apache.spark.sql.sources.{CreateTableUsing, CreateTempTableUsing, DescribeCommand => LogicalDescribeCommand, _} import org.apache.spark.sql.types._ -import org.apache.spark.sql.sources.{DescribeCommand => LogicalDescribeCommand} -import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand} -import org.apache.spark.sql.sources._ +import org.apache.spark.sql.{SQLContext, Strategy, execution} private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { self: SQLContext#SparkPlanner => 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 8372decbf8aa1..f27585d05a986 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 @@ -27,7 +27,7 @@ import org.apache.spark.sql.types.StructType private[sql] class DefaultSource - extends RelationProvider with SchemaRelationProvider with CreateableRelationProvider { + extends RelationProvider with SchemaRelationProvider with CreatableRelationProvider { /** Returns a new base relation with the parameters. */ override def createRelation( 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 14c81ae4eba4e..19bfba34b8f4a 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 @@ -159,7 +159,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { val attributesSize = attributes.size if (attributesSize > record.size) { throw new IndexOutOfBoundsException( - s"Trying to write more fields than contained in row (${attributesSize}>${record.size})") + s"Trying to write more fields than contained in row ($attributesSize > ${record.size})") } var index = 0 @@ -325,7 +325,7 @@ private[parquet] class MutableRowWriteSupport extends RowWriteSupport { val attributesSize = attributes.size if (attributesSize > record.size) { throw new IndexOutOfBoundsException( - s"Trying to write more fields than contained in row (${attributesSize}>${record.size})") + s"Trying to write more fields than contained in row ($attributesSize > ${record.size})") } var index = 0 @@ -348,10 +348,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[String].getBytes("utf-8"))) 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/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index b646109b7c553..5209581fa8357 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 @@ -19,24 +19,23 @@ package org.apache.spark.sql.parquet import java.io.IOException +import scala.collection.mutable.ArrayBuffer import scala.util.Try import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapreduce.Job - import parquet.format.converter.ParquetMetadataConverter -import parquet.hadoop.{ParquetFileReader, Footer, ParquetFileWriter} -import parquet.hadoop.metadata.{ParquetMetadata, FileMetaData} +import parquet.hadoop.metadata.{FileMetaData, ParquetMetadata} import parquet.hadoop.util.ContextUtil -import parquet.schema.{Type => ParquetType, Types => ParquetTypes, PrimitiveType => ParquetPrimitiveType, MessageType} -import parquet.schema.{GroupType => ParquetGroupType, OriginalType => ParquetOriginalType, ConversionPatterns, DecimalMetadata} +import parquet.hadoop.{Footer, ParquetFileReader, ParquetFileWriter} import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName} import parquet.schema.Type.Repetition +import parquet.schema.{ConversionPatterns, DecimalMetadata, GroupType => ParquetGroupType, MessageType, OriginalType => ParquetOriginalType, PrimitiveType => ParquetPrimitiveType, Type => ParquetType, Types => ParquetTypes} -import org.apache.spark.Logging -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.types._ +import org.apache.spark.{Logging, SparkException} // Implicits import scala.collection.JavaConversions._ @@ -285,7 +284,7 @@ private[parquet] object ParquetTypesConverter extends Logging { ctype: DataType, name: String, nullable: Boolean = true, - inArray: Boolean = false, + inArray: Boolean = false, toThriftSchemaNames: Boolean = false): ParquetType = { val repetition = if (inArray) { @@ -340,7 +339,7 @@ private[parquet] object ParquetTypesConverter extends Logging { } case StructType(structFields) => { val fields = structFields.map { - field => fromDataType(field.dataType, field.name, field.nullable, + field => fromDataType(field.dataType, field.name, field.nullable, inArray = false, toThriftSchemaNames) } new ParquetGroupType(repetition, name, fields.toSeq) 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 179c0d6b22239..49d46334b6525 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 @@ -16,25 +16,38 @@ */ package org.apache.spark.sql.parquet -import java.util.{List => JList} +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 scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer +import scala.util.Try -import org.apache.hadoop.conf.{Configurable, Configuration} +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} 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.hadoop.ParquetInputFormat +import parquet.format.converter.ParquetMetadataConverter +import parquet.hadoop.{ParquetInputFormat, _} import parquet.hadoop.util.ContextUtil import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.rdd.{NewHadoopPartition, RDD} +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.parquet.ParquetTypesConverter._ import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types.{IntegerType, StructField, StructType} -import org.apache.spark.sql.{Row, SQLConf, SQLContext} -import org.apache.spark.{Logging, Partition => SparkPartition} +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} /** @@ -43,19 +56,49 @@ import org.apache.spark.{Logging, Partition => SparkPartition} * required is `path`, which should be the location of a collection of, optionally partitioned, * parquet files. */ -class DefaultSource extends RelationProvider { +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.")) + } + /** Returns a new base relation with the given parameters. */ override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - val path = - parameters.getOrElse("path", sys.error("'path' must be specified for parquet tables.")) + ParquetRelation2(Seq(checkPath(parameters)), parameters, None)(sqlContext) + } - ParquetRelation2(path)(sqlContext) + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String], + schema: StructType): BaseRelation = { + ParquetRelation2(Seq(checkPath(parameters)), parameters, Some(schema))(sqlContext) + } + + override def createRelation( + sqlContext: SQLContext, + 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) + relation } } -private[parquet] case class Partition(partitionValues: Map[String, Any], files: Seq[FileStatus]) +private[parquet] case class Partition(values: Row, path: String) + +private[parquet] case class PartitionSpec(partitionColumns: StructType, partitions: Seq[Partition]) /** * An alternative to [[ParquetRelation]] that plugs in using the data sources API. This class is @@ -81,117 +124,196 @@ private[parquet] case class Partition(partitionValues: Map[String, Any], files: * discovery. */ @DeveloperApi -case class ParquetRelation2(path: String)(@transient val sqlContext: SQLContext) - extends CatalystScan with Logging { +case class ParquetRelation2 + (paths: Seq[String], parameters: Map[String, String], maybeSchema: Option[StructType] = None) + (@transient val sqlContext: SQLContext) + extends CatalystScan + with InsertableRelation + with SparkHadoopMapReduceUtil + with Logging { + + // Should we merge schemas from all Parquet part-files? + private val shouldMergeSchemas = + parameters.getOrElse(ParquetRelation2.MERGE_SCHEMA, "true").toBoolean + + // Optional Metastore schema, used when converting Hive Metastore Parquet table + private val maybeMetastoreSchema = + parameters + .get(ParquetRelation2.METASTORE_SCHEMA) + .map(s => DataType.fromJson(s).asInstanceOf[StructType]) + + // Hive uses this as part of the default partition name when the partition column value is null + // or empty string + private val defaultPartitionName = parameters.getOrElse( + ParquetRelation2.DEFAULT_PARTITION_NAME, "__HIVE_DEFAULT_PARTITION__") + + override def equals(other: Any) = other match { + case relation: ParquetRelation2 => + paths.toSet == relation.paths.toSet && + maybeMetastoreSchema == relation.maybeMetastoreSchema && + (shouldMergeSchemas == relation.shouldMergeSchemas || schema == relation.schema) + } - def sparkContext = sqlContext.sparkContext + private[sql] def sparkContext = sqlContext.sparkContext - // Minor Hack: scala doesnt seem to respect @transient for vals declared via extraction - @transient - private var partitionKeys: Seq[String] = _ - @transient - private var partitions: Seq[Partition] = _ - discoverPartitions() + @transient private val fs = FileSystem.get(sparkContext.hadoopConfiguration) - // TODO: Only finds the first partition, assumes the key is of type Integer... - private def discoverPartitions() = { - val fs = FileSystem.get(new java.net.URI(path), sparkContext.hadoopConfiguration) - val partValue = "([^=]+)=([^=]+)".r + private class MetadataCache { + private var metadataStatuses: Array[FileStatus] = _ + private var commonMetadataStatuses: Array[FileStatus] = _ + private var footers: Map[FileStatus, Footer] = _ + private var parquetSchema: StructType = _ - val childrenOfPath = fs.listStatus(new Path(path)).filterNot(_.getPath.getName.startsWith("_")) - val childDirs = childrenOfPath.filter(s => s.isDir) + var dataStatuses: Array[FileStatus] = _ + var partitionSpec: PartitionSpec = _ + var schema: StructType = _ + var dataSchemaIncludesPartitionKeys: Boolean = _ - if (childDirs.size > 0) { - val partitionPairs = childDirs.map(_.getPath.getName).map { - case partValue(key, value) => (key, value) + 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 foundKeys = partitionPairs.map(_._1).distinct - if (foundKeys.size > 1) { - sys.error(s"Too many distinct partition keys: $foundKeys") + val leaves = baseStatuses.flatMap { f => + val statuses = 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 } - // Do a parallel lookup of partition metadata. - val partitionFiles = - childDirs.par.map { d => - fs.listStatus(d.getPath) - // TODO: Is there a standard hadoop function for this? - .filterNot(_.getPath.getName.startsWith("_")) - .filterNot(_.getPath.getName.startsWith(".")) - }.seq - - partitionKeys = foundKeys.toSeq - partitions = partitionFiles.zip(partitionPairs).map { case (files, (key, value)) => - Partition(Map(key -> value.toInt), files) - }.toSeq - } else { - partitionKeys = Nil - partitions = Partition(Map.empty, childrenOfPath) :: Nil - } - } + dataStatuses = leaves.filterNot(f => isSummaryFile(f.getPath)) + metadataStatuses = leaves.filter(_.getPath.getName == ParquetFileWriter.PARQUET_METADATA_FILE) + commonMetadataStatuses = + leaves.filter(_.getPath.getName == ParquetFileWriter.PARQUET_COMMON_METADATA_FILE) + + footers = (dataStatuses ++ metadataStatuses ++ commonMetadataStatuses).par.map { f => + val parquetMetadata = ParquetFileReader.readFooter( + sparkContext.hadoopConfiguration, f, ParquetMetadataConverter.NO_FILTER) + f -> new Footer(f.getPath, parquetMetadata) + }.seq.toMap + + partitionSpec = { + val partitionDirs = dataStatuses + .filterNot(baseStatuses.contains) + .map(_.getPath.getParent) + .distinct + + if (partitionDirs.nonEmpty) { + ParquetRelation2.parsePartitions(partitionDirs, defaultPartitionName) + } else { + // No partition directories found, makes an empty specification + PartitionSpec(StructType(Seq.empty[StructField]), Seq.empty[Partition]) + } + } - override val sizeInBytes = partitions.flatMap(_.files).map(_.getLen).sum + parquetSchema = maybeSchema.getOrElse(readSchema()) - val dataSchema = StructType.fromAttributes( // TODO: Parquet code should not deal with attributes. - ParquetTypesConverter.readSchemaFromFile( - partitions.head.files.head.getPath, - Some(sparkContext.hadoopConfiguration), - sqlContext.conf.isParquetBinaryAsString, - sqlContext.conf.isParquetINT96AsTimestamp)) + dataSchemaIncludesPartitionKeys = + isPartitioned && + partitionColumns.forall(f => metadataCache.parquetSchema.fieldNames.contains(f.name)) - val dataIncludesKey = - partitionKeys.headOption.map(dataSchema.fieldNames.contains(_)).getOrElse(true) + schema = { + val fullParquetSchema = if (dataSchemaIncludesPartitionKeys) { + metadataCache.parquetSchema + } else { + StructType(metadataCache.parquetSchema.fields ++ partitionColumns.fields) + } - override val schema = - if (dataIncludesKey) { - dataSchema - } else { - StructType(dataSchema.fields :+ StructField(partitionKeys.head, IntegerType)) + maybeMetastoreSchema + .map(ParquetRelation2.mergeMetastoreParquetSchema(_, fullParquetSchema)) + .getOrElse(fullParquetSchema) + } } - override def buildScan(output: Seq[Attribute], predicates: Seq[Expression]): RDD[Row] = { - // This is mostly a hack so that we can use the existing parquet filter code. - val requiredColumns = output.map(_.name) + private def readSchema(): 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, + // "_common_metadata" is more preferable than "_metadata" because it doesn't contain row + // groups information, and could be much smaller for large Parquet files with lots of row + // groups. + // + // NOTE: Metadata stored in the summary files are merged from all part-files. However, for + // user defined key-value metadata (in which we store Spark SQL schema), Parquet doesn't know + // how to merge them correctly if some key is associated with different values in different + // part-files. When this happens, Parquet simply gives up generating the summary file. This + // implies that if a summary file presents, then: + // + // 1. Either all part-files have exactly the same Spark SQL schema, or + // 2. Some part-files don't contain Spark SQL schema in the key-value metadata at all (thus + // their schemas may differ from each other). + // + // Here we tend to be pessimistic and take the second case into account. Basically this means + // we can't trust the summary files if users require a merged schema, and must touch all part- + // files to do the merge. + if (shouldMergeSchemas) { + // Also includes summary files, 'cause there might be empty partition directories. + (metadataStatuses ++ commonMetadataStatuses ++ dataStatuses).toSeq + } else { + // Tries any "_common_metadata" first. Parquet files written by old versions or Parquet + // don't have this. + commonMetadataStatuses.headOption + // Falls back to "_metadata" + .orElse(metadataStatuses.headOption) + // Summary file(s) not found, the Parquet file is either corrupted, or different part- + // files contain conflicting user defined metadata (two or more values are associated + // with a same key in different files). In either case, we fall back to any of the + // first part-file, and just assume all schemas are consistent. + .orElse(dataStatuses.headOption) + .toSeq + } - val job = new Job(sparkContext.hadoopConfiguration) - ParquetInputFormat.setReadSupportClass(job, classOf[RowReadSupport]) - val jobConf: Configuration = ContextUtil.getConfiguration(job) + ParquetRelation2.readSchema(filesToTouch.map(footers.apply), sqlContext) + } + } - val requestedSchema = StructType(requiredColumns.map(schema(_))) + @transient private val metadataCache = new MetadataCache + metadataCache.refresh() - val partitionKeySet = partitionKeys.toSet - val rawPredicate = - predicates - .filter(_.references.map(_.name).toSet.subsetOf(partitionKeySet)) - .reduceOption(And) - .getOrElse(Literal(true)) + private def partitionColumns = metadataCache.partitionSpec.partitionColumns - // Translate the predicate so that it reads from the information derived from the - // folder structure - val castedPredicate = rawPredicate transform { - case a: AttributeReference => - val idx = partitionKeys.indexWhere(a.name == _) - BoundReference(idx, IntegerType, nullable = true) - } + private def partitions = metadataCache.partitionSpec.partitions - val inputData = new GenericMutableRow(partitionKeys.size) - val pruningCondition = InterpretedPredicate(castedPredicate) + private def isPartitioned = partitionColumns.nonEmpty - val selectedPartitions = - if (partitionKeys.nonEmpty && predicates.nonEmpty) { - partitions.filter { part => - inputData(0) = part.partitionValues.values.head - pruningCondition(inputData) - } - } else { - partitions + private def dataSchemaIncludesPartitionKeys = metadataCache.dataSchemaIncludesPartitionKeys + + override def schema = metadataCache.schema + + private def isSummaryFile(file: Path): Boolean = { + file.getName == ParquetFileWriter.PARQUET_COMMON_METADATA_FILE || + file.getName == ParquetFileWriter.PARQUET_METADATA_FILE + } + + // 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). + override val sizeInBytes = metadataCache.dataStatuses.map(_.getLen).sum + + // This is mostly a hack so that we can use the existing parquet filter code. + override def buildScan(output: Seq[Attribute], predicates: Seq[Expression]): RDD[Row] = { + val job = new Job(sparkContext.hadoopConfiguration) + ParquetInputFormat.setReadSupportClass(job, classOf[RowReadSupport]) + val jobConf: Configuration = ContextUtil.getConfiguration(job) + + val selectedPartitions = prunePartitions(predicates, partitions) + val selectedFiles = if (isPartitioned) { + selectedPartitions.flatMap { p => + metadataCache.dataStatuses.filter(_.getPath.getParent.toString == p.path) } + } else { + metadataCache.dataStatuses.toSeq + } - val fs = FileSystem.get(new java.net.URI(path), sparkContext.hadoopConfiguration) - val selectedFiles = selectedPartitions.flatMap(_.files).map(f => fs.makeQualified(f.getPath)) // FileInputFormat cannot handle empty lists. if (selectedFiles.nonEmpty) { - org.apache.hadoop.mapreduce.lib.input.FileInputFormat.setInputPaths(job, selectedFiles: _*) + FileInputFormat.setInputPaths(job, selectedFiles.map(_.getPath): _*) } // Push down filters when possible. Notice that not all filters can be converted to Parquet @@ -203,23 +325,28 @@ case class ParquetRelation2(path: String)(@transient val sqlContext: SQLContext) .filter(_ => sqlContext.conf.parquetFilterPushDown) .foreach(ParquetInputFormat.setFilterPredicate(jobConf, _)) - def percentRead = selectedPartitions.size.toDouble / partitions.size.toDouble * 100 - logInfo(s"Reading $percentRead% of $path partitions") + if (isPartitioned) { + def percentRead = selectedPartitions.size.toDouble / partitions.size.toDouble * 100 + logInfo(s"Reading $percentRead% of partitions") + } + + val requiredColumns = output.map(_.name) + val requestedSchema = StructType(requiredColumns.map(schema(_))) // Store both requested and original schema in `Configuration` jobConf.set( RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA, - ParquetTypesConverter.convertToString(requestedSchema.toAttributes)) + convertToString(requestedSchema.toAttributes)) jobConf.set( RowWriteSupport.SPARK_ROW_SCHEMA, - ParquetTypesConverter.convertToString(schema.toAttributes)) + convertToString(schema.toAttributes)) // Tell FilteringParquetRowInputFormat whether it's okay to cache Parquet and FS metadata val useCache = sqlContext.getConf(SQLConf.PARQUET_CACHE_METADATA, "true").toBoolean jobConf.set(SQLConf.PARQUET_CACHE_METADATA, useCache.toString) val baseRDD = - new org.apache.spark.rdd.NewHadoopRDD( + new NewHadoopRDD( sparkContext, classOf[FilteringParquetRowInputFormat], classOf[Void], @@ -228,66 +355,400 @@ case class ParquetRelation2(path: String)(@transient val sqlContext: SQLContext) val cacheMetadata = useCache @transient - val cachedStatus = selectedPartitions.flatMap(_.files) + val cachedStatus = selectedFiles // 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 - } - } else { - new FilteringParquetRowInputFormat + val inputFormat = if (cacheMetadata) { + new FilteringParquetRowInputFormat { + override def listStatus(jobContext: JobContext): JList[FileStatus] = cachedStatus } - - inputFormat match { - case configurable: Configurable => - configurable.setConf(getConf) - case _ => + } else { + new FilteringParquetRowInputFormat } + val jobContext = newJobContext(getConf, jobId) - val rawSplits = inputFormat.getSplits(jobContext).toArray - val result = new Array[SparkPartition](rawSplits.size) - for (i <- 0 until rawSplits.size) { - result(i) = - new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) + val rawSplits = inputFormat.getSplits(jobContext) + + Array.tabulate[SparkPartition](rawSplits.size) { i => + new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) } - result } } - // The ordinal for the partition key in the result row, if requested. - val partitionKeyLocation = - partitionKeys - .headOption - .map(requiredColumns.indexOf(_)) - .getOrElse(-1) + // The ordinals for partition keys in the result row, if requested. + val partitionKeyLocations = partitionColumns.fieldNames.zipWithIndex.map { + case (name, index) => index -> requiredColumns.indexOf(name) + }.toMap.filter { + case (_, index) => index >= 0 + } // 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 (!dataIncludesKey && partitionKeyLocation != -1) { - baseRDD.mapPartitionsWithInputSplit { case (split, iter) => - val partValue = "([^=]+)=([^=]+)".r - val partValues = - split.asInstanceOf[parquet.hadoop.ParquetInputSplit] - .getPath - .toString - .split("/") - .flatMap { - case partValue(key, value) => Some(key -> value) - case _ => None - }.toMap - - val currentValue = partValues.values.head.toInt - iter.map { pair => - val res = pair._2.asInstanceOf[SpecificMutableRow] - res.setInt(partitionKeyLocation, currentValue) - res + if (!dataSchemaIncludesPartitionKeys && partitionKeyLocations.nonEmpty) { + baseRDD.mapPartitionsWithInputSplit { case (split: ParquetInputSplit, iterator) => + val partValues = selectedPartitions.collectFirst { + case p if split.getPath.getParent.toString == p.path => p.values + }.get + + iterator.map { pair => + val row = pair._2.asInstanceOf[SpecificMutableRow] + var i = 0 + while (i < partValues.size) { + // TODO Avoids boxing cost here! + row.update(partitionKeyLocations(i), partValues(i)) + i += 1 + } + row } } } else { baseRDD.map(_._2) } } + + private def prunePartitions( + predicates: Seq[Expression], + partitions: Seq[Partition]): Seq[Partition] = { + val partitionColumnNames = partitionColumns.map(_.name).toSet + val partitionPruningPredicates = predicates.filter { + _.references.map(_.name).toSet.subsetOf(partitionColumnNames) + } + + val rawPredicate = partitionPruningPredicates.reduceOption(And).getOrElse(Literal(true)) + val boundPredicate = InterpretedPredicate(rawPredicate transform { + case a: AttributeReference => + val index = partitionColumns.indexWhere(a.name == _.name) + BoundReference(index, partitionColumns(index).dataType, nullable = true) + }) + + if (isPartitioned && partitionPruningPredicates.nonEmpty) { + partitions.filter(p => boundPredicate(p.values)) + } else { + partitions + } + } + + override def insert(data: DataFrame, overwrite: Boolean): Unit = { + // 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 + // to catch this early one, maybe having the planner validate the schema + // before calling execute(). + + val job = new Job(sqlContext.sparkContext.hadoopConfiguration) + val writeSupport = if (schema.map(_.dataType).forall(_.isPrimitive)) { + log.debug("Initializing MutableRowWriteSupport") + classOf[MutableRowWriteSupport] + } else { + classOf[RowWriteSupport] + } + + ParquetOutputFormat.setWriteSupportClass(job, writeSupport) + + val conf = ContextUtil.getConfiguration(job) + RowWriteSupport.setSchema(schema.toAttributes, conf) + + val destinationPath = new Path(paths.head) + + if (overwrite) { + try { + destinationPath.getFileSystem(conf).delete(destinationPath, true) + } catch { + case e: IOException => + throw new IOException( + s"Unable to clear output directory ${destinationPath.toString} prior" + + s" to writing to Parquet file:\n${e.toString}") + } + } + + job.setOutputKeyClass(classOf[Void]) + job.setOutputValueClass(classOf[Row]) + FileOutputFormat.setOutputPath(job, destinationPath) + + val wrappedConf = new SerializableWritable(job.getConfiguration) + val jobTrackerId = new SimpleDateFormat("yyyyMMddHHmm").format(new Date()) + val stageId = sqlContext.sparkContext.newRddId() + + val taskIdOffset = if (overwrite) { + 1 + } else { + FileSystemHelper.findMaxTaskId( + FileOutputFormat.getOutputPath(job).toString, job.getConfiguration) + 1 + } + + def writeShard(context: TaskContext, iterator: Iterator[Row]): Unit = { + /* "reduce task" */ + val attemptId = newTaskAttemptID( + jobTrackerId, stageId, isMap = false, context.partitionId(), context.attemptNumber()) + val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) + val format = new AppendingParquetOutputFormat(taskIdOffset) + val committer = format.getOutputCommitter(hadoopContext) + committer.setupTask(hadoopContext) + val writer = format.getRecordWriter(hadoopContext) + try { + while (iterator.hasNext) { + val row = iterator.next() + writer.write(null, row) + } + } finally { + writer.close(hadoopContext) + } + committer.commitTask(hadoopContext) + } + val jobFormat = new AppendingParquetOutputFormat(taskIdOffset) + /* apparently we need a TaskAttemptID to construct an OutputCommitter; + * however we're only going to use this local OutputCommitter for + * setupJob/commitJob, so we just use a dummy "map" task. + */ + val jobAttemptId = newTaskAttemptID(jobTrackerId, stageId, isMap = true, 0, 0) + val jobTaskContext = newTaskAttemptContext(wrappedConf.value, jobAttemptId) + val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext) + + jobCommitter.setupJob(jobTaskContext) + sqlContext.sparkContext.runJob(data.queryExecution.executedPlan.execute(), writeShard _) + jobCommitter.commitJob(jobTaskContext) + + metadataCache.refresh() + } +} + +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 + val DEFAULT_PARTITION_NAME = "partition.defaultName" + + // When true, the Parquet data source caches Parquet metadata for performance + val CACHE_METADATA = "cacheMetadata" + + private[parquet] def readSchema(footers: Seq[Footer], sqlContext: SQLContext): StructType = { + footers.map { footer => + val metadata = footer.getParquetMetadata.getFileMetaData + val parquetSchema = metadata.getSchema + val maybeSparkSchema = metadata + .getKeyValueMetaData + .toMap + .get(RowReadSupport.SPARK_METADATA_KEY) + .map(DataType.fromJson(_).asInstanceOf[StructType]) + + maybeSparkSchema.getOrElse { + // Falls back to Parquet schema if Spark SQL schema is absent. + StructType.fromAttributes( + // TODO Really no need to use `Attribute` here, we only need to know the data type. + convertToAttributes( + parquetSchema, + sqlContext.conf.isParquetBinaryAsString, + sqlContext.conf.isParquetINT96AsTimestamp)) + } + }.reduce { (left, right) => + try left.merge(right) catch { case e: Throwable => + throw new SparkException(s"Failed to merge incompatible schemas $left and $right", e) + } + } + } + + private[parquet] def mergeMetastoreParquetSchema( + metastoreSchema: StructType, + parquetSchema: StructType): StructType = { + def schemaConflictMessage = + s"""Converting Hive Metastore Parquet, but detected conflicting schemas. Metastore schema: + |${metastoreSchema.prettyJson} + | + |Parquet schema: + |${parquetSchema.prettyJson} + """.stripMargin + + 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)) + + StructType(metastoreSchema.zip(reorderedParquetSchema).map { + // Uses Parquet field names but retains Metastore data types. + case (mSchema, pSchema) if mSchema.name.toLowerCase == pSchema.name.toLowerCase => + mSchema.copy(name = pSchema.name) + case _ => + throw new SparkException(schemaConflictMessage) + }) + } + + // 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... + private[parquet] case class PartitionValues(columnNames: Seq[String], literals: Seq[Literal]) { + require(columnNames.size == literals.size) + } + + /** + * Given a group of qualified paths, tries to parse them and returns a partition specification. + * For example, given: + * {{{ + * hdfs://:/path/to/partition/a=1/b=hello/c=3.14 + * hdfs://:/path/to/partition/a=2/b=world/c=6.28 + * }}} + * it returns: + * {{{ + * PartitionSpec( + * partitionColumns = StructType( + * StructField(name = "a", dataType = IntegerType, nullable = true), + * StructField(name = "b", dataType = StringType, nullable = true), + * StructField(name = "c", dataType = DoubleType, nullable = true)), + * partitions = Seq( + * Partition( + * values = Row(1, "hello", 3.14), + * path = "hdfs://:/path/to/partition/a=1/b=hello/c=3.14"), + * Partition( + * values = Row(2, "world", 6.28), + * path = "hdfs://:/path/to/partition/a=2/b=world/c=6.28"))) + * }}} + */ + private[parquet] def parsePartitions( + paths: Seq[Path], + defaultPartitionName: String): PartitionSpec = { + val partitionValues = resolvePartitions(paths.map(parsePartition(_, defaultPartitionName))) + val fields = { + val (PartitionValues(columnNames, literals)) = partitionValues.head + columnNames.zip(literals).map { case (name, Literal(_, dataType)) => + StructField(name, dataType, nullable = true) + } + } + + val partitions = partitionValues.zip(paths).map { + case (PartitionValues(_, literals), path) => + Partition(Row(literals.map(_.value): _*), path.toString) + } + + PartitionSpec(StructType(fields), partitions) + } + + /** + * Parses a single partition, returns column names and values of each partition column. For + * example, given: + * {{{ + * path = hdfs://:/path/to/partition/a=42/b=hello/c=3.14 + * }}} + * it returns: + * {{{ + * PartitionValues( + * Seq("a", "b", "c"), + * Seq( + * Literal(42, IntegerType), + * Literal("hello", StringType), + * Literal(3.14, FloatType))) + * }}} + */ + private[parquet] def parsePartition( + path: Path, + defaultPartitionName: String): PartitionValues = { + val columns = ArrayBuffer.empty[(String, Literal)] + // Old Hadoop versions don't have `Path.isRoot` + var finished = path.getParent == null + var chopped = path + + while (!finished) { + val maybeColumn = parsePartitionColumn(chopped.getName, defaultPartitionName) + maybeColumn.foreach(columns += _) + chopped = chopped.getParent + finished = maybeColumn.isEmpty || chopped.getParent == null + } + + val (columnNames, values) = columns.reverse.unzip + PartitionValues(columnNames, values) + } + + private def parsePartitionColumn( + columnSpec: String, + defaultPartitionName: String): Option[(String, Literal)] = { + val equalSignIndex = columnSpec.indexOf('=') + if (equalSignIndex == -1) { + None + } else { + val columnName = columnSpec.take(equalSignIndex) + assert(columnName.nonEmpty, s"Empty partition column name in '$columnSpec'") + + val rawColumnValue = columnSpec.drop(equalSignIndex + 1) + assert(rawColumnValue.nonEmpty, s"Empty partition column value in '$columnSpec'") + + val literal = inferPartitionColumnValue(rawColumnValue, defaultPartitionName) + Some(columnName -> literal) + } + } + + /** + * Resolves possible type conflicts between partitions by up-casting "lower" types. The up- + * casting order is: + * {{{ + * NullType -> + * IntegerType -> LongType -> + * FloatType -> DoubleType -> DecimalType.Unlimited -> + * StringType + * }}} + */ + 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", "") + s"Conflicting partition column names detected:\n$list" + }) + + // Resolves possible type conflicts for each column + val resolvedValues = (0 until columnCount).map { i => + resolveTypeConflicts(values.map(_.literals(i))) + } + + // Fills resolved literals back to each partition + values.zipWithIndex.map { case (d, index) => + d.copy(literals = resolvedValues.map(_(index))) + } + } + + /** + * Converts a string to a `Literal` with automatic type inference. Currently only supports + * [[IntegerType]], [[LongType]], [[FloatType]], [[DoubleType]], [[DecimalType.Unlimited]], and + * [[StringType]]. + */ + private[parquet] def inferPartitionColumnValue( + raw: String, + defaultPartitionName: String): Literal = { + // First tries integral types + Try(Literal(Integer.parseInt(raw), IntegerType)) + .orElse(Try(Literal(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))) + // Then falls back to string + .getOrElse { + if (raw == defaultPartitionName) Literal(null, NullType) else Literal(raw, StringType) + } + } + + private val upCastingOrder: Seq[DataType] = + Seq(NullType, IntegerType, LongType, FloatType, DoubleType, DecimalType.Unlimited, StringType) + + /** + * Given a collection of [[Literal]]s, resolves possible type conflicts by up-casting "lower" + * types. + */ + private def resolveTypeConflicts(literals: Seq[Literal]): Seq[Literal] = { + val desiredType = { + val topType = literals.map(_.dataType).maxBy(upCastingOrder.indexOf(_)) + // Falls back to string if all values of this column are null or empty string + if (topType == NullType) StringType else topType + } + + literals.map { case l @ Literal(_, dataType) => + Literal(Cast(l, desiredType).eval(), desiredType) + } + } } 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 386ff2452f1a3..d23ffb8b7a960 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 @@ -18,12 +18,12 @@ package org.apache.spark.sql.sources import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, Strategy} 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.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, InsertIntoTable => LogicalInsertIntoTable} -import org.apache.spark.sql.execution +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} /** * A Strategy for planning scans over data sources defined using the sources API. @@ -54,7 +54,7 @@ private[sql] object DataSourceStrategy extends Strategy { case l @ LogicalRelation(t: TableScan) => execution.PhysicalRDD(l.output, t.buildScan()) :: Nil - case i @ LogicalInsertIntoTable( + 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.") 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 2ef740b3be0bd..9c37e0169ff85 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 @@ -241,20 +241,16 @@ object ResolvedDataSource { val relation = userSpecifiedSchema match { case Some(schema: StructType) => { clazz.newInstance match { - case dataSource: org.apache.spark.sql.sources.SchemaRelationProvider => - dataSource - .asInstanceOf[org.apache.spark.sql.sources.SchemaRelationProvider] - .createRelation(sqlContext, new CaseInsensitiveMap(options), schema) + 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: org.apache.spark.sql.sources.RelationProvider => - dataSource - .asInstanceOf[org.apache.spark.sql.sources.RelationProvider] - .createRelation(sqlContext, new CaseInsensitiveMap(options)) + 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}.") } @@ -279,10 +275,8 @@ object ResolvedDataSource { } val relation = clazz.newInstance match { - case dataSource: org.apache.spark.sql.sources.CreateableRelationProvider => - dataSource - .asInstanceOf[org.apache.spark.sql.sources.CreateableRelationProvider] - .createRelation(sqlContext, options, data) + case dataSource: CreatableRelationProvider => + dataSource.createRelation(sqlContext, options, data) case _ => sys.error(s"${clazz.getCanonicalName} does not allow create table as select.") } @@ -366,7 +360,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 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 ad0a35b91ebc2..40fc1f2aa2724 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 @@ -78,7 +78,7 @@ trait SchemaRelationProvider { } @DeveloperApi -trait CreateableRelationProvider { +trait CreatableRelationProvider { def createRelation( sqlContext: SQLContext, parameters: Map[String, String], 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 ff91a0eb42049..f8117c21773ae 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,8 +22,10 @@ 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.types._ +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.sources.LogicalRelation import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.types._ import org.apache.spark.sql.{Column, DataFrame, QueryTest, SQLConf} /** @@ -54,9 +56,17 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { .select(output.map(e => Column(e)): _*) .where(Column(predicate)) - val maybeAnalyzedPredicate = query.queryExecution.executedPlan.collect { - case plan: ParquetTableScan => plan.columnPruningPred - }.flatten.reduceOption(_ && _) + val maybeAnalyzedPredicate = { + val forParquetTableScan = query.queryExecution.executedPlan.collect { + case plan: ParquetTableScan => plan.columnPruningPred + }.flatten.reduceOption(_ && _) + + val forParquetDataSource = query.queryExecution.optimizedPlan.collect { + case PhysicalOperation(_, filters, LogicalRelation(_: ParquetRelation2)) => filters + }.flatten.reduceOption(_ && _) + + forParquetTableScan.orElse(forParquetDataSource) + } assert(maybeAnalyzedPredicate.isDefined) maybeAnalyzedPredicate.foreach { pred => @@ -84,213 +94,228 @@ class ParquetFilterSuite extends QueryTest with ParquetTest { checkFilterPredicate(predicate, filterClass, Seq(Row(expected)))(rdd) } - 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) + private def checkBinaryFilterPredicate + (predicate: Predicate, filterClass: Class[_ <: FilterPredicate], expected: Seq[Row]) + (implicit rdd: DataFrame): Unit = { + def checkBinaryAnswer(rdd: DataFrame, expected: Seq[Row]) = { + assertResult(expected.map(_.getAs[Array[Byte]](0).mkString(",")).toSeq.sorted) { + rdd.map(_.getAs[Array[Byte]](0).mkString(",")).collect().toSeq.sorted + } } + + checkFilterPredicate(rdd, predicate, filterClass, checkBinaryAnswer _, expected) } - 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))) - } + private def checkBinaryFilterPredicate + (predicate: Predicate, filterClass: Class[_ <: FilterPredicate], expected: Array[Byte]) + (implicit rdd: DataFrame): Unit = { + checkBinaryFilterPredicate(predicate, filterClass, Seq(Row(expected)))(rdd) } - 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(_))) + 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))) + + 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(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(_))) - 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("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(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(_))) - 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("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(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(_))) - 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("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(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(_))) - 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("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 - 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"))) + } } - } - def checkBinaryFilterPredicate - (predicate: Predicate, filterClass: Class[_ <: FilterPredicate], expected: Seq[Row]) - (implicit rdd: DataFrame): Unit = { - def checkBinaryAnswer(rdd: DataFrame, expected: Seq[Row]) = { - assertResult(expected.map(_.getAs[Array[Byte]](0).mkString(",")).toSeq.sorted) { - rdd.map(_.getAs[Array[Byte]](0).mkString(",")).collect().toSeq.sorted + test(s"$prefix: filter pushdown - binary") { + implicit class IntToBinary(int: Int) { + def b: Array[Byte] = int.toString.getBytes("UTF-8") } - } - checkFilterPredicate(rdd, predicate, filterClass, checkBinaryAnswer _, expected) - } + withParquetRDD((1 to 4).map(i => Tuple1(i.b))) { implicit rdd => + checkBinaryFilterPredicate('_1 === 1.b, classOf[Eq[_]], 1.b) - def checkBinaryFilterPredicate - (predicate: Predicate, filterClass: Class[_ <: FilterPredicate], expected: Array[Byte]) - (implicit rdd: DataFrame): Unit = { - checkBinaryFilterPredicate(predicate, filterClass, Seq(Row(expected)))(rdd) - } + checkBinaryFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkBinaryFilterPredicate( + '_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(i => Row.apply(i.b)).toSeq) - test("filter pushdown - binary") { - implicit class IntToBinary(int: Int) { - def b: Array[Byte] = int.toString.getBytes("UTF-8") - } + 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) - withParquetRDD((1 to 4).map(i => Tuple1(i.b))) { implicit rdd => - 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[Eq [_]], 1.b) - 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(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(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))) + } } } + + withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API -> "true") { + run("Parquet data source enabled") + } + + withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API -> "false") { + run("Parquet data source disabled") + } } 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 0bc246c645602..c8ebbbc7d2eac 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,218 +73,229 @@ class ParquetIOSuite extends QueryTest with ParquetTest { withParquetRDD(data)(r => checkAnswer(r, data.map(Row.fromTuple))) } - test("basic data types (without binary)") { - val data = (1 to 4).map { i => - (i % 2 == 0, i, i.toLong, i.toFloat, i.toDouble) + 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) } - checkParquetFile(data) - } - 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: 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("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("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)) - .select($"_1" cast decimal as "abcd") + 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)) + } - 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(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)) + // 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(s"$prefix: 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: array") { + val data = (1 to 4).map(i => Tuple1(Seq(i, i + 1))) + 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: 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 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: 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("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 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("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: 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("nones") { - val allNones = ( - None.asInstanceOf[Option[Int]], - None.asInstanceOf[Option[Long]], - None.asInstanceOf[Option[String]]) + test(s"$prefix: 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): _*)) + withParquetRDD(allNones :: Nil) { rdd => + val rows = rdd.collect() + assert(rows.size === 1) + assert(rows.head === Row(Seq.fill(3)(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: 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 + } - val data = (0 until 10).map(i => (i, i.toString)) + val data = (0 until 10).map(i => (i, i.toString)) - def checkCompressionCodec(codec: CompressionCodecName): Unit = { - withSQLConf(SQLConf.PARQUET_COMPRESSION -> codec.name()) { - withParquetFile(data) { path => - assertResult(conf.parquetCompressionCodec.toUpperCase) { - compressionCodecFor(path) + def checkCompressionCodec(codec: CompressionCodecName): Unit = { + withSQLConf(SQLConf.PARQUET_COMPRESSION -> codec.name()) { + withParquetFile(data) { path => + assertResult(conf.parquetCompressionCodec.toUpperCase) { + compressionCodecFor(path) + } } } } - } - // Checks default compression codec - checkCompressionCodec(CompressionCodecName.fromConf(conf.parquetCompressionCodec)) + // Checks default compression codec + checkCompressionCodec(CompressionCodecName.fromConf(conf.parquetCompressionCodec)) - checkCompressionCodec(CompressionCodecName.UNCOMPRESSED) - checkCompressionCodec(CompressionCodecName.GZIP) - checkCompressionCodec(CompressionCodecName.SNAPPY) - } + checkCompressionCodec(CompressionCodecName.UNCOMPRESSED) + checkCompressionCodec(CompressionCodecName.GZIP) + checkCompressionCodec(CompressionCodecName.SNAPPY) + } - 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) - } + 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) + } - writer.close() - } + writer.close() + } - 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) - }) + 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("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(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) - assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_COMMON_METADATA_FILE))) - assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE))) + assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_COMMON_METADATA_FILE))) + assert(fs.exists(new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE))) - val metaData = ParquetTypesConverter.readMetaData(path, Some(configuration)) - val actualSchema = metaData.getFileMetaData.getSchema - val expectedSchema = ParquetTypesConverter.convertFromAttributes(attributes) + val metaData = ParquetTypesConverter.readMetaData(path, Some(configuration)) + val actualSchema = metaData.getFileMetaData.getSchema + val expectedSchema = ParquetTypesConverter.convertFromAttributes(attributes) - actualSchema.checkContains(expectedSchema) - expectedSchema.checkContains(actualSchema) + actualSchema.checkContains(expectedSchema) + expectedSchema.checkContains(actualSchema) + } } } + + withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API -> "true") { + run("Parquet data source enabled") + } + + withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API -> "false") { + run("Parquet data source disabled") + } } 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 new file mode 100644 index 0000000000000..ae606d11a8f68 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetPartitionDiscoverySuite.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.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} + +class ParquetPartitionDiscoverySuite extends FunSuite with ParquetTest { + override val sqlContext: SQLContext = TestSQLContext + + val defaultPartitionName = "__NULL__" + + test("column type inference") { + def check(raw: String, literal: Literal): Unit = { + 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)) + } + + test("parse partition") { + def check(path: String, expected: PartitionValues): Unit = { + assert(expected === parsePartition(new Path(path), defaultPartitionName)) + } + + def checkThrows[T <: Throwable: Manifest](path: String, expected: String): Unit = { + val message = intercept[T] { + parsePartition(new Path(path), defaultPartitionName) + }.getMessage + + assert(message.contains(expected)) + } + + check( + "file:///", + PartitionValues( + ArrayBuffer.empty[String], + ArrayBuffer.empty[Literal])) + + check( + "file://path/a=10", + PartitionValues( + ArrayBuffer("a"), + ArrayBuffer(Literal(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)))) + + check( + "file://path/a=10/b_hello/c=1.5", + PartitionValues( + ArrayBuffer("c"), + ArrayBuffer(Literal(1.5, FloatType)))) + + checkThrows[AssertionError]("file://path/=10", "Empty partition column name") + checkThrows[AssertionError]("file://path/a=", "Empty partition column value") + } + + test("parse partitions") { + def check(paths: Seq[String], spec: PartitionSpec): Unit = { + assert(parsePartitions(paths.map(new Path(_)), defaultPartitionName) === spec) + } + + check(Seq( + "hdfs://host:9000/path/a=10/b=hello"), + PartitionSpec( + StructType(Seq( + StructField("a", IntegerType), + StructField("b", StringType))), + Seq(Partition(Row(10, "hello"), "hdfs://host:9000/path/a=10/b=hello")))) + + check(Seq( + "hdfs://host:9000/path/a=10/b=20", + "hdfs://host:9000/path/a=10.5/b=hello"), + PartitionSpec( + StructType(Seq( + StructField("a", FloatType), + StructField("b", StringType))), + Seq( + 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=$defaultPartitionName", + s"hdfs://host:9000/path/a=10.5/b=$defaultPartitionName"), + PartitionSpec( + StructType(Seq( + StructField("a", FloatType), + StructField("b", StringType))), + Seq( + 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")))) + } +} 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 5ec7a156d9353..48c7598343e55 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,10 +17,10 @@ package org.apache.spark.sql.parquet -import org.apache.spark.sql.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. @@ -28,82 +28,93 @@ import org.apache.spark.sql.test.TestSQLContext._ class ParquetQuerySuite extends QueryTest with ParquetTest { val sqlContext = TestSQLContext - 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(_))) + 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("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)) + // TODO Re-enable this after data source insertion API is merged + 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)) + } } - } - // This test case will trigger the NPE mentioned in - // https://issues.apache.org/jira/browse/PARQUET-151. - 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)) + // 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)) + } } - } - 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) - } + 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 + withParquetTable(data, "t") { + val selfJoin = sql("SELECT * FROM t x JOIN t y WHERE x._1 = y._1") + val queryOutput = selfJoin.queryExecution.analyzed.output - 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 - } + 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 + } - checkAnswer(selfJoin, List(Row(1, "1", 1, "1"), Row(3, "3", 3, "3"))) + checkAnswer(selfJoin, List(Row(1, "1", 1, "1"), Row(3, "3", 3, "3"))) + } } - } - test("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(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 - 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(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("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(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-5309 strings stored using dictionary compression in parquet") { - withParquetTable((0 until 1000).map(i => ("same", "run_" + i /100, 1)), "t") { + test(s"$prefix: 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(s"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(s"SELECT _1, _2, SUM(_3) FROM t WHERE _2 = 'run_5' GROUP BY _1, _2"), + List(Row("same", "run_5", 100))) + } } } + + withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API -> "true") { + run("Parquet data source enabled") + } + + withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API -> "false") { + run("Parquet data source disabled") + } } 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 5f7f31d395cf7..2e6c2d5f9ab55 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 @@ -25,6 +25,7 @@ import parquet.schema.MessageTypeParser import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.types._ class ParquetSchemaSuite extends FunSuite with ParquetTest { val sqlContext = TestSQLContext @@ -192,4 +193,40 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest { assert(a.nullable === b.nullable) } } + + test("merge with metastore schema") { + // Field type conflict resolution + assertResult( + StructType(Seq( + StructField("lowerCase", StringType), + StructField("UPPERCase", DoubleType, nullable = false)))) { + + ParquetRelation2.mergeMetastoreParquetSchema( + StructType(Seq( + StructField("lowercase", StringType), + StructField("uppercase", DoubleType, nullable = false))), + + 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))), + + StructType(Seq( + StructField("lowerCase", BinaryType), + StructField("UPPERCase", IntegerType, nullable = true)))) + }.getMessage.contains("detected conflicting schemas")) + + // Conflicting field names + intercept[Throwable] { + ParquetRelation2.mergeMetastoreParquetSchema( + StructType(Seq(StructField("lower", StringType))), + StructType(Seq(StructField("lowerCase", BinaryType)))) + } + } } 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 243310686d08a..c78369d12cf55 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 @@ -38,6 +38,7 @@ import org.apache.spark.sql.catalyst.expressions._ 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.sources.{DDLParser, LogicalRelation, ResolvedDataSource} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -175,10 +176,25 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Nil } - // Since HiveQL is case insensitive for table names we make them all lowercase. - MetastoreRelation( + 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) + } + + LogicalRelation(ParquetRelation2( + paths, Map(ParquetRelation2.METASTORE_SCHEMA -> metastoreSchema.json))(hive)) + } else { + relation + } } } 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 7857a0252ebb3..95abc363ae767 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 @@ -87,7 +87,8 @@ private[hive] trait HiveStrategies { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case PhysicalOperation(projectList, predicates, relation: MetastoreRelation) if relation.tableDesc.getSerdeClassName.contains("Parquet") && - hiveContext.convertMetastoreParquet => + hiveContext.convertMetastoreParquet && + !hiveContext.conf.parquetUseDataSourceApi => // Filter out all predicates that only deal with partition keys val partitionsKeys = AttributeSet(relation.partitionKeys) @@ -136,8 +137,10 @@ private[hive] trait HiveStrategies { pruningCondition(inputData) } + val partitionLocations = partitions.map(_.getLocation) + hiveContext - .parquetFile(partitions.map(_.getLocation).mkString(",")) + .parquetFile(partitionLocations.head, partitionLocations.tail: _*) .addPartitioningAttributes(relation.partitionKeys) .lowerCase .where(unresolvedOtherPredicates) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala index 581f666399492..eae69af5864aa 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -28,53 +28,55 @@ class HiveParquetSuite extends QueryTest with ParquetTest { import sqlContext._ - test("Case insensitive attribute names") { - withParquetTable((1 to 4).map(i => Cases(i.toString, i.toString)), "cases") { - val expected = (1 to 4).map(i => Row(i.toString)) - checkAnswer(sql("SELECT upper FROM cases"), expected) - checkAnswer(sql("SELECT LOWER FROM cases"), expected) + def run(prefix: String): Unit = { + test(s"$prefix: Case insensitive attribute names") { + withParquetTable((1 to 4).map(i => Cases(i.toString, i.toString)), "cases") { + val expected = (1 to 4).map(i => Row(i.toString)) + checkAnswer(sql("SELECT upper FROM cases"), expected) + checkAnswer(sql("SELECT LOWER FROM cases"), expected) + } } - } - test("SELECT on Parquet table") { - val data = (1 to 4).map(i => (i, s"val_$i")) - withParquetTable(data, "t") { - checkAnswer(sql("SELECT * FROM t"), data.map(Row.fromTuple)) - } - } - - test("Simple column projection + filter on Parquet table") { - withParquetTable((1 to 4).map(i => (i % 2 == 0, i, s"val_$i")), "t") { - checkAnswer( - sql("SELECT `_1`, `_3` FROM t WHERE `_1` = true"), - Seq(Row(true, "val_2"), Row(true, "val_4"))) + test(s"$prefix: SELECT on Parquet table") { + val data = (1 to 4).map(i => (i, s"val_$i")) + withParquetTable(data, "t") { + checkAnswer(sql("SELECT * FROM t"), data.map(Row.fromTuple)) + } } - } - test("Converting Hive to Parquet Table via saveAsParquetFile") { - withTempPath { dir => - sql("SELECT * FROM src").saveAsParquetFile(dir.getCanonicalPath) - parquetFile(dir.getCanonicalPath).registerTempTable("p") - withTempTable("p") { + test(s"$prefix: Simple column projection + filter on Parquet table") { + withParquetTable((1 to 4).map(i => (i % 2 == 0, i, s"val_$i")), "t") { checkAnswer( - sql("SELECT * FROM src ORDER BY key"), - sql("SELECT * from p ORDER BY key").collect().toSeq) + sql("SELECT `_1`, `_3` FROM t WHERE `_1` = true"), + Seq(Row(true, "val_2"), Row(true, "val_4"))) } } - } - - test("INSERT OVERWRITE TABLE Parquet table") { - withParquetTable((1 to 4).map(i => (i, s"val_$i")), "t") { - withTempPath { file => - sql("SELECT * FROM t LIMIT 1").saveAsParquetFile(file.getCanonicalPath) - parquetFile(file.getCanonicalPath).registerTempTable("p") + test(s"$prefix: Converting Hive to Parquet Table via saveAsParquetFile") { + withTempPath { dir => + sql("SELECT * FROM src").saveAsParquetFile(dir.getCanonicalPath) + parquetFile(dir.getCanonicalPath).registerTempTable("p") withTempTable("p") { - // let's do three overwrites for good measure - sql("INSERT OVERWRITE TABLE p SELECT * FROM t") - sql("INSERT OVERWRITE TABLE p SELECT * FROM t") - sql("INSERT OVERWRITE TABLE p SELECT * FROM t") - checkAnswer(sql("SELECT * FROM p"), sql("SELECT * FROM t").collect().toSeq) + checkAnswer( + sql("SELECT * FROM src ORDER BY key"), + sql("SELECT * from p ORDER BY key").collect().toSeq) + } + } + } + + // TODO Re-enable this after data source insertion API is merged + ignore(s"$prefix: INSERT OVERWRITE TABLE Parquet table") { + withParquetTable((1 to 10).map(i => (i, s"val_$i")), "t") { + withTempPath { file => + sql("SELECT * FROM t LIMIT 1").saveAsParquetFile(file.getCanonicalPath) + parquetFile(file.getCanonicalPath).registerTempTable("p") + withTempTable("p") { + // let's do three overwrites for good measure + sql("INSERT OVERWRITE TABLE p SELECT * FROM t") + sql("INSERT OVERWRITE TABLE p SELECT * FROM t") + sql("INSERT OVERWRITE TABLE p SELECT * FROM t") + checkAnswer(sql("SELECT * FROM p"), sql("SELECT * FROM t").collect().toSeq) + } } } } 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 30441bbbdf817..a7479a5b95864 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 @@ -23,7 +23,8 @@ import java.io.File import org.apache.spark.sql.catalyst.expressions.Row import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.QueryTest +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._ @@ -79,7 +80,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' - location '${new File(partitionedTableDir, "p=1").getCanonicalPath}' + location '${new File(normalTableDir, "normal").getCanonicalPath}' """) (1 to 10).foreach { p => @@ -97,7 +98,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { setConf("spark.sql.hive.convertMetastoreParquet", "false") } - test("conversion is working") { + test(s"conversion is working") { assert( sql("SELECT * FROM normal_parquet").queryExecution.executedPlan.collect { case _: HiveTableScan => true @@ -105,6 +106,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { assert( sql("SELECT * FROM normal_parquet").queryExecution.executedPlan.collect { case _: ParquetTableScan => true + case _: PhysicalRDD => true }.nonEmpty) } } @@ -147,6 +149,7 @@ class ParquetSourceSuite extends ParquetPartitioningTest { */ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll { var partitionedTableDir: File = null + var normalTableDir: File = null var partitionedTableDirWithKey: File = null import org.apache.spark.sql.hive.test.TestHive.implicits._ @@ -156,6 +159,10 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll partitionedTableDir.delete() partitionedTableDir.mkdir() + normalTableDir = File.createTempFile("parquettests", "sparksql") + normalTableDir.delete() + normalTableDir.mkdir() + (1 to 10).foreach { p => val partDir = new File(partitionedTableDir, s"p=$p") sparkContext.makeRDD(1 to 10) @@ -163,6 +170,11 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll .saveAsParquetFile(partDir.getCanonicalPath) } + sparkContext + .makeRDD(1 to 10) + .map(i => ParquetData(i, s"part-1")) + .saveAsParquetFile(new File(normalTableDir, "normal").getCanonicalPath) + partitionedTableDirWithKey = File.createTempFile("parquettests", "sparksql") partitionedTableDirWithKey.delete() partitionedTableDirWithKey.mkdir() @@ -175,99 +187,107 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll } } - 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)) + 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) + } } - test(s"pruned count $table") { + test(s"$prefix: $prefix: non-part select(*)") { checkAnswer( - sql(s"SELECT COUNT(*) FROM $table WHERE p = 1"), + sql("SELECT COUNT(*) FROM normal_parquet"), Row(10)) } - - test(s"non-existant 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)) - } - - 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)) - } + setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "false") + run("Parquet data source enabled") + + setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "true") + run("Parquet data source disabled") } From 85ccee81acef578ec4b40fb5f5d97b9e24314f35 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 5 Feb 2015 16:26:51 -0800 Subject: [PATCH 100/578] [SPARK-5620][DOC] group methods in generated unidoc It seems that `(ScalaUnidoc, unidoc)` is the correct way to overwrite `scalacOptions` in unidoc. CC: rxin gzm0 Author: Xiangrui Meng Closes #4404 from mengxr/SPARK-5620 and squashes the following commits: f890cf5 [Xiangrui Meng] add -groups to scalacOptions in unidoc --- project/SparkBuild.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 93698efe84252..f63f9c1982bb5 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -374,7 +374,10 @@ object Unidoc { ), "-group", "Spark SQL", packageList("sql.api.java", "sql.api.java.types", "sql.hive.api.java"), "-noqualifier", "java.lang" - ) + ), + + // Group similar methods together based on the @group annotation. + scalacOptions in (ScalaUnidoc, unidoc) ++= Seq("-groups") ) } From e8a5d50a96f6e7d4fce33ea19fbfc083f4351296 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 5 Feb 2015 18:07:10 -0800 Subject: [PATCH 101/578] [SPARK-5638][SQL] Add a config flag to disable eager analysis of DataFrames Author: Reynold Xin Closes #4408 from rxin/df-config-eager and squashes the following commits: c0204cf [Reynold Xin] [SPARK-5638][SQL] Add a config flag to disable eager analysis of DataFrames. --- .../org/apache/spark/sql/DataFrameImpl.scala | 4 +++- .../scala/org/apache/spark/sql/SQLConf.scala | 6 ++++++ .../org/apache/spark/sql/DataFrameSuite.scala | 17 ++++++++++++++--- 3 files changed, 23 insertions(+), 4 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 58d11751353b3..4911443dd6dde 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 @@ -53,7 +53,9 @@ private[sql] class DataFrameImpl protected[sql]( def this(sqlContext: SQLContext, logicalPlan: LogicalPlan) = { this(sqlContext, { val qe = sqlContext.executePlan(logicalPlan) - qe.analyzed // This should force analysis and throw errors if there are any + if (sqlContext.conf.dataFrameEagerAnalysis) { + qe.analyzed // 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 5ef3bb022fc5a..180f5e765fb91 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,6 +52,9 @@ private[spark] object SQLConf { // This is used to set the default data source val DEFAULT_DATA_SOURCE_NAME = "spark.sql.default.datasource" + // Whether to perform eager analysis on a DataFrame. + val DATAFRAME_EAGER_ANALYSIS = "spark.sql.dataframe.eagerAnalysis" + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" } @@ -173,6 +176,9 @@ private[sql] class SQLConf extends Serializable { private[spark] def defaultDataSourceName: String = getConf(DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.parquet") + private[spark] def dataFrameEagerAnalysis: Boolean = + getConf(DATAFRAME_EAGER_ANALYSIS, "true").toBoolean + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ 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 74c29459d2e47..77fd3165f151f 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,19 +17,23 @@ package org.apache.spark.sql +import scala.language.postfixOps + import org.apache.spark.sql.Dsl._ import org.apache.spark.sql.types._ - -/* Implicits */ +import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.logicalPlanToSparkQuery import org.apache.spark.sql.test.TestSQLContext.implicits._ -import scala.language.postfixOps class DataFrameSuite extends QueryTest { import org.apache.spark.sql.TestData._ test("analysis error should be eagerly reported") { + val oldSetting = TestSQLContext.conf.dataFrameEagerAnalysis + // Eager analysis. + TestSQLContext.setConf(SQLConf.DATAFRAME_EAGER_ANALYSIS, "true") + intercept[Exception] { testData.select('nonExistentName) } intercept[Exception] { testData.groupBy('key).agg(Map("nonExistentName" -> "sum")) @@ -40,6 +44,13 @@ class DataFrameSuite extends QueryTest { intercept[Exception] { testData.groupBy($"abcd").agg(Map("key" -> "sum")) } + + // No more eager analysis once the flag is turned off + TestSQLContext.setConf(SQLConf.DATAFRAME_EAGER_ANALYSIS, "false") + testData.select('nonExistentName) + + // Set the flag back to original value before this test. + TestSQLContext.setConf(SQLConf.DATAFRAME_EAGER_ANALYSIS, oldSetting.toString) } test("table scan") { From 7c0a648fb5537ba7a1fe2545ead49219b14b656c Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 5 Feb 2015 18:09:18 -0800 Subject: [PATCH 102/578] [HOTFIX] [SQL] Disables Metastore Parquet table conversion for "SQLQuerySuite.CTAS with serde" Ideally we should convert Metastore Parquet tables with our own Parquet implementation on both read path and write path. However, the write path is not well covered, and causes this test failure. This PR is a hotfix to bring back Jenkins PR builder. A proper fix will be delivered in a follow-up PR. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4413) Author: Cheng Lian Closes #4413 from liancheng/hotfix-parquet-ctas and squashes the following commits: 5291289 [Cheng Lian] Hot fix for "SQLQuerySuite.CTAS with serde" --- .../sql/hive/execution/SQLQuerySuite.scala | 57 ++++++++++--------- 1 file changed, 30 insertions(+), 27 deletions(-) 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 22310ffadd25e..49fe79d989259 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,13 +17,10 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.QueryTest - -import org.apache.spark.sql.Row +import org.apache.spark.sql.hive.HiveShim import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.types._ -import org.apache.spark.util.Utils -import org.apache.spark.sql.hive.HiveShim +import org.apache.spark.sql.{QueryTest, Row, SQLConf} case class Nested1(f1: Nested2) case class Nested2(f2: Nested3) @@ -109,28 +106,34 @@ class SQLQuerySuite extends QueryTest { ) if (HiveShim.version =="0.13.1") { - sql( - """CREATE TABLE ctas5 - | STORED AS parquet AS - | SELECT key, value - | FROM src - | ORDER BY key, value""".stripMargin).collect - - checkExistence(sql("DESC EXTENDED ctas5"), true, - "name:key", "type:string", "name:value", "ctas5", - "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat", - "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat", - "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe", - "MANAGED_TABLE" - ) - - val default = getConf("spark.sql.hive.convertMetastoreParquet", "true") - // use the Hive SerDe for parquet tables - sql("set spark.sql.hive.convertMetastoreParquet = false") - checkAnswer( - sql("SELECT key, value FROM ctas5 ORDER BY key, value"), - sql("SELECT key, value FROM src ORDER BY key, value").collect().toSeq) - sql(s"set spark.sql.hive.convertMetastoreParquet = $default") + val origUseParquetDataSource = conf.parquetUseDataSourceApi + try { + setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "false") + sql( + """CREATE TABLE ctas5 + | STORED AS parquet AS + | SELECT key, value + | FROM src + | ORDER BY key, value""".stripMargin).collect() + + checkExistence(sql("DESC EXTENDED ctas5"), true, + "name:key", "type:string", "name:value", "ctas5", + "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat", + "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat", + "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe", + "MANAGED_TABLE" + ) + + val default = getConf("spark.sql.hive.convertMetastoreParquet", "true") + // use the Hive SerDe for parquet tables + sql("set spark.sql.hive.convertMetastoreParquet = false") + checkAnswer( + sql("SELECT key, value FROM ctas5 ORDER BY key, value"), + sql("SELECT key, value FROM src ORDER BY key, value").collect().toSeq) + sql(s"set spark.sql.hive.convertMetastoreParquet = $default") + } finally { + setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, origUseParquetDataSource.toString) + } } } From 793dbaef401d777c3efc1759a3ea7580e01de528 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 5 Feb 2015 18:14:54 -0800 Subject: [PATCH 103/578] SPARK-5557: Explicitly include servlet API in dependencies. Because of the way we shade jetty, we lose its dependency orbit in the assembly jar, which includes the javax servlet API's. This adds back orbit explicitly, using the version that matches our jetty version. Author: Patrick Wendell Closes #4411 from pwendell/servlet-api and squashes the following commits: 445f868 [Patrick Wendell] SPARK-5557: Explicitly include servlet API in dependencies. --- core/pom.xml | 7 +++++++ pom.xml | 1 + 2 files changed, 8 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index 2dc5f747f2b71..4daaf88147142 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -132,6 +132,13 @@ jetty-servlet compile + + + org.eclipse.jetty.orbit + javax.servlet + ${orbit.version} + org.apache.commons diff --git a/pom.xml b/pom.xml index d324b5f0ec93a..727cd0a36b0af 100644 --- a/pom.xml +++ b/pom.xml @@ -135,6 +135,7 @@ 1.6.0rc3 1.2.3 8.1.14.v20131031 + 3.0.0.v201112011016 0.5.0 2.24.0 2.4.0 From 6d3b7cbee82b72c695447c13f9f73bcc594b2caf Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 5 Feb 2015 18:36:48 -0800 Subject: [PATCH 104/578] Revert "SPARK-5607: Update to Kryo 2.24.0 to avoid including objenesis 1.2." This reverts commit c3b8d272cf0574e72422d8d7f4f0683dcbdce41b. --- pom.xml | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/pom.xml b/pom.xml index 727cd0a36b0af..aef450ae63121 100644 --- a/pom.xml +++ b/pom.xml @@ -137,7 +137,6 @@ 8.1.14.v20131031 3.0.0.v201112011016 0.5.0 - 2.24.0 2.4.0 2.0.8 3.1.0 @@ -342,13 +341,7 @@ - - - com.esotericsoftware.kryo - kryo - ${kryo.version} - - + From 7dc4965f34e37b37f4fab69859fcce6476f87811 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 5 Feb 2015 23:02:40 -0800 Subject: [PATCH 105/578] [SPARK-5639][SQL] Support DataFrame.renameColumn. Author: Reynold Xin Closes #4410 from rxin/df-renameCol and squashes the following commits: a6a796e [Reynold Xin] [SPARK-5639][SQL] Support DataFrame.renameColumn. --- .../org/apache/spark/sql/DataFrame.scala | 9 +++++++- .../org/apache/spark/sql/DataFrameImpl.scala | 8 +++++++ .../apache/spark/sql/IncomputableColumn.scala | 2 ++ .../org/apache/spark/sql/DataFrameSuite.scala | 21 +++++++++++++++++++ 4 files changed, 39 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 92e04ce17c2e7..8ad6526f872e5 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 @@ -36,7 +36,8 @@ private[sql] object DataFrame { /** - * A collection of rows that have the same columns. + * :: 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]]. @@ -72,6 +73,7 @@ private[sql] object DataFrame { * }}} */ // TODO: Improve documentation. +@Experimental trait DataFrame extends RDDApi[Row] { val sqlContext: SQLContext @@ -425,6 +427,11 @@ trait DataFrame extends RDDApi[Row] { */ def addColumn(colName: String, col: Column): DataFrame + /** + * Returns a new [[DataFrame]] with a column renamed. + */ + def renameColumn(existingName: String, newName: String): DataFrame + /** * Returns the first `n` rows. */ 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 4911443dd6dde..789bcf6184b3e 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,6 +238,14 @@ private[sql] class DataFrameImpl protected[sql]( 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 head(n: Int): Array[Row] = limit(n).collect() override def head(): Row = head(1).head 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 fedd7f06ef50a..6043fb4dee01d 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 @@ -108,6 +108,8 @@ 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() 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 77fd3165f151f..5aa3db720c886 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 @@ -313,6 +313,27 @@ class DataFrameSuite extends QueryTest { ) } + test("addColumn") { + val df = testData.toDataFrame.addColumn("newCol", col("key") + 1) + checkAnswer( + df, + testData.collect().map { case Row(key: Int, value: String) => + Row(key, value, key + 1) + }.toSeq) + assert(df.schema.map(_.name).toSeq === Seq("key", "value", "newCol")) + } + + test("renameColumn") { + val df = testData.toDataFrame.addColumn("newCol", col("key") + 1) + .renameColumn("value", "valueRenamed") + checkAnswer( + df, + testData.collect().map { case Row(key: Int, value: String) => + Row(key, value, key + 1) + }.toSeq) + assert(df.schema.map(_.name).toSeq === Seq("key", "valueRenamed", "newCol")) + } + test("apply on query results (SPARK-5462)") { val df = testData.sqlContext.sql("select key from testData") checkAnswer(df("key"), testData.select('key).collect().toSeq) From 6b88825a25a0a072c13bbcc57bbfdb102a3f133d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 5 Feb 2015 23:32:09 -0800 Subject: [PATCH 106/578] [SPARK-5604][MLLIB] remove checkpointDir from trees This is the second part of SPARK-5604, which removes checkpointDir from tree strategies. Note that this is a break change. I will mention it in the migration guide. Author: Xiangrui Meng Closes #4407 from mengxr/SPARK-5604-1 and squashes the following commits: 13a276d [Xiangrui Meng] remove checkpointDir from trees --- .../spark/examples/mllib/DecisionTreeRunner.scala | 3 ++- .../org/apache/spark/mllib/tree/RandomForest.scala | 1 - .../spark/mllib/tree/configuration/Strategy.scala | 10 ++++------ .../apache/spark/mllib/tree/impl/NodeIdCache.scala | 12 ------------ 4 files changed, 6 insertions(+), 20 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 205d80dd02682..262fd2c9611d0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -272,6 +272,8 @@ object DecisionTreeRunner { case Variance => impurity.Variance } + params.checkpointDir.foreach(sc.setCheckpointDir) + val strategy = new Strategy( algo = params.algo, @@ -282,7 +284,6 @@ object DecisionTreeRunner { minInstancesPerNode = params.minInstancesPerNode, minInfoGain = params.minInfoGain, useNodeIdCache = params.useNodeIdCache, - checkpointDir = params.checkpointDir, checkpointInterval = params.checkpointInterval) if (params.numTrees == 1) { val startTime = System.nanoTime() 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 45b0154c5e4cb..db01f2e229e5a 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 @@ -204,7 +204,6 @@ private class RandomForest ( Some(NodeIdCache.init( data = baggedInput, numTrees = numTrees, - checkpointDir = strategy.checkpointDir, checkpointInterval = strategy.checkpointInterval, initVal = 1)) } else { 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 3308adb6752ff..8d5c36da32bdb 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 @@ -62,11 +62,10 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * @param subsamplingRate Fraction of the training data used for learning decision tree. * @param useNodeIdCache If this is true, instead of passing trees to executors, the algorithm will * maintain a separate RDD of node Id cache for each row. - * @param checkpointDir If the node Id cache is used, it will help to checkpoint - * the node Id cache periodically. This is the checkpoint directory - * to be used for the node Id cache. * @param checkpointInterval How often to checkpoint when the node Id cache gets updated. - * E.g. 10 means that the cache will get checkpointed every 10 updates. + * E.g. 10 means that the cache will get checkpointed every 10 updates. If + * the checkpoint directory is not set in + * [[org.apache.spark.SparkContext]], this setting is ignored. */ @Experimental class Strategy ( @@ -82,7 +81,6 @@ class Strategy ( @BeanProperty var maxMemoryInMB: Int = 256, @BeanProperty var subsamplingRate: Double = 1, @BeanProperty var useNodeIdCache: Boolean = false, - @BeanProperty var checkpointDir: Option[String] = None, @BeanProperty var checkpointInterval: Int = 10) extends Serializable { def isMulticlassClassification = @@ -165,7 +163,7 @@ class Strategy ( def copy: Strategy = { new Strategy(algo, impurity, maxDepth, numClasses, maxBins, quantileCalculationStrategy, categoricalFeaturesInfo, minInstancesPerNode, minInfoGain, - maxMemoryInMB, subsamplingRate, useNodeIdCache, checkpointDir, checkpointInterval) + maxMemoryInMB, subsamplingRate, useNodeIdCache, checkpointInterval) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/NodeIdCache.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/NodeIdCache.scala index 83011b48b7d9b..bdd0f576b048d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/NodeIdCache.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/NodeIdCache.scala @@ -71,15 +71,12 @@ private[tree] case class NodeIndexUpdater( * The nodeIdsForInstances RDD needs to be updated at each iteration. * @param nodeIdsForInstances The initial values in the cache * (should be an Array of all 1's (meaning the root nodes)). - * @param checkpointDir The checkpoint directory where - * the checkpointed files will be stored. * @param checkpointInterval The checkpointing interval * (how often should the cache be checkpointed.). */ @DeveloperApi private[tree] class NodeIdCache( var nodeIdsForInstances: RDD[Array[Int]], - val checkpointDir: Option[String], val checkpointInterval: Int) { // Keep a reference to a previous node Ids for instances. @@ -91,12 +88,6 @@ private[tree] class NodeIdCache( private val checkpointQueue = mutable.Queue[RDD[Array[Int]]]() private var rddUpdateCount = 0 - // If a checkpoint directory is given, and there's no prior checkpoint directory, - // then set the checkpoint directory with the given one. - if (checkpointDir.nonEmpty && nodeIdsForInstances.sparkContext.getCheckpointDir.isEmpty) { - nodeIdsForInstances.sparkContext.setCheckpointDir(checkpointDir.get) - } - /** * Update the node index values in the cache. * This updates the RDD and its lineage. @@ -184,7 +175,6 @@ private[tree] object NodeIdCache { * Initialize the node Id cache with initial node Id values. * @param data The RDD of training rows. * @param numTrees The number of trees that we want to create cache for. - * @param checkpointDir The checkpoint directory where the checkpointed files will be stored. * @param checkpointInterval The checkpointing interval * (how often should the cache be checkpointed.). * @param initVal The initial values in the cache. @@ -193,12 +183,10 @@ private[tree] object NodeIdCache { def init( data: RDD[BaggedPoint[TreePoint]], numTrees: Int, - checkpointDir: Option[String], checkpointInterval: Int, initVal: Int = 1): NodeIdCache = { new NodeIdCache( data.map(_ => Array.fill[Int](numTrees)(initVal)), - checkpointDir, checkpointInterval) } } From dc0c4490a12ecedd8ca5a1bb256c7ccbdf0be04f Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 5 Feb 2015 23:43:47 -0800 Subject: [PATCH 107/578] [SPARK-4789] [SPARK-4942] [SPARK-5031] [mllib] Standardize ML Prediction APIs MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This is part (1a) of the updates from the design doc in [https://docs.google.com/document/d/1BH9el33kBX8JiDdgUJXdLW14CA2qhTCWIG46eXZVoJs] **UPDATE**: Most of the APIs are being kept private[spark] to allow further discussion. Here is a list of changes which are public: * new output columns: rawPrediction, probabilities * The “score” column is now called “rawPrediction” * Classifiers now provide numClasses * Params.get and .set are now protected instead of private[ml]. * ParamMap now has a size method. * new classes: LinearRegression, LinearRegressionModel * LogisticRegression now has an intercept. ### Sketch of APIs (most of which are private[spark] for now) Abstract classes for learning algorithms (+ corresponding Model abstractions): * Classifier (+ ClassificationModel) * ProbabilisticClassifier (+ ProbabilisticClassificationModel) * Regressor (+ RegressionModel) * Predictor (+ PredictionModel) * *For all of these*: * There is no strongly typed training-time API. * There is a strongly typed test-time (prediction) API which helps developers implement new algorithms. Concrete classes: learning algorithms * LinearRegression * LogisticRegression (updated to use new abstract classes) * Also, removed "score" in favor of "probability" output column. Changed BinaryClassificationEvaluator to match. (SPARK-5031) Other updates: * params.scala: Changed Params.set/get to be protected instead of private[ml] * This was needed for the example of defining a class from outside of the MLlib namespace. * VectorUDT: Will later change from private[spark] to public. * This is needed for outside users to write their own validateAndTransformSchema() methods using vectors. * Also, added equals() method.f * SPARK-4942 : ML Transformers should allow output cols to be turned on,off * Update validateAndTransformSchema * Update transform * (Updated examples, test suites according to other changes) New examples: * DeveloperApiExample.scala (example of defining algorithm from outside of the MLlib namespace) * Added Java version too Test Suites: * LinearRegressionSuite * LogisticRegressionSuite * + Java versions of above suites CC: mengxr etrain shivaram Author: Joseph K. Bradley Closes #3637 from jkbradley/ml-api-part1 and squashes the following commits: 405bfb8 [Joseph K. Bradley] Last edits based on code review. Small cleanups fec348a [Joseph K. Bradley] Added JavaDeveloperApiExample.java and fixed other issues: Made developer API private[spark] for now. Added constructors Java can understand to specialized Param types. 8316d5e [Joseph K. Bradley] fixes after rebasing on master fc62406 [Joseph K. Bradley] fixed test suites after last commit bcb9549 [Joseph K. Bradley] Fixed issues after rebasing from master (after move from SchemaRDD to DataFrame) 9872424 [Joseph K. Bradley] fixed JavaLinearRegressionSuite.java Java sql api f542997 [Joseph K. Bradley] Added MIMA excludes for VectorUDT (now public), and added DeveloperApi annotation to it 216d199 [Joseph K. Bradley] fixed after sql datatypes PR got merged f549e34 [Joseph K. Bradley] Updates based on code review. Major ones are: * Created weakly typed Predictor.train() method which is called by fit() so that developers do not have to call schema validation or copy parameters. * Made Predictor.featuresDataType have a default value of VectorUDT. * NOTE: This could be dangerous since the FeaturesType type parameter cannot have a default value. 343e7bd [Joseph K. Bradley] added blanket mima exclude for ml package 82f340b [Joseph K. Bradley] Fixed bug in LogisticRegression (introduced in this PR). Fixed Java suites 0a16da9 [Joseph K. Bradley] Fixed Linear/Logistic RegressionSuites c3c8da5 [Joseph K. Bradley] small cleanup 934f97b [Joseph K. Bradley] Fixed bugs from previous commit. 1c61723 [Joseph K. Bradley] * Made ProbabilisticClassificationModel into a subclass of ClassificationModel. Also introduced ProbabilisticClassifier. * This was to support output column “probabilityCol” in transform(). 4e2f711 [Joseph K. Bradley] rat fix bc654e1 [Joseph K. Bradley] Added spark.ml LinearRegressionSuite 8d13233 [Joseph K. Bradley] Added methods: * Classifier: batch predictRaw() * Predictor: train() without paramMap ProbabilisticClassificationModel.predictProbabilities() * Java versions of all above batch methods + others 1680905 [Joseph K. Bradley] Added JavaLabeledPointSuite.java for spark.ml, and added constructor to LabeledPoint which defaults weight to 1.0 adbe50a [Joseph K. Bradley] * fixed LinearRegression train() to use embedded paramMap * added Predictor.predict(RDD[Vector]) method * updated Linear/LogisticRegressionSuites 58802e3 [Joseph K. Bradley] added train() to Predictor subclasses which does not take a ParamMap. 57d54ab [Joseph K. Bradley] * Changed semantics of Predictor.train() to merge the given paramMap with the embedded paramMap. * remove threshold_internal from logreg * Added Predictor.copy() * Extended LogisticRegressionSuite e433872 [Joseph K. Bradley] Updated docs. Added LabeledPointSuite to spark.ml 54b7b31 [Joseph K. Bradley] Fixed issue with logreg threshold being set correctly 0617d61 [Joseph K. Bradley] Fixed bug from last commit (sorting paramMap by parameter names in toString). Fixed bug in persisting logreg data. Added threshold_internal to logreg for faster test-time prediction (avoiding map lookup). 601e792 [Joseph K. Bradley] Modified ParamMap to sort parameters in toString. Cleaned up classes in class hierarchy, before implementing tests and examples. d705e87 [Joseph K. Bradley] Added LinearRegression and Regressor back from ml-api branch 52f4fde [Joseph K. Bradley] removing everything except for simple class hierarchy for classification d35bb5d [Joseph K. Bradley] fixed compilation issues, but have not added tests yet bfade12 [Joseph K. Bradley] Added lots of classes for new ML API: --- .../ml/JavaCrossValidatorExample.java | 6 +- .../examples/ml/JavaDeveloperApiExample.java | 217 ++++++++++++++++ .../examples/ml/JavaSimpleParamsExample.java | 10 +- .../JavaSimpleTextClassificationPipeline.java | 4 +- .../examples/ml/CrossValidatorExample.scala | 7 +- .../examples/ml/DeveloperApiExample.scala | 184 ++++++++++++++ .../examples/ml/SimpleParamsExample.scala | 16 +- .../ml/SimpleTextClassificationPipeline.scala | 7 +- .../scala/org/apache/spark/ml/Estimator.scala | 9 +- .../spark/ml/classification/Classifier.scala | 206 +++++++++++++++ .../classification/LogisticRegression.scala | 212 ++++++++++------ .../ProbabilisticClassifier.scala | 147 +++++++++++ .../BinaryClassificationEvaluator.scala | 24 +- .../apache/spark/ml/feature/Tokenizer.scala | 4 +- .../spark/ml/impl/estimator/Predictor.scala | 234 ++++++++++++++++++ .../org/apache/spark/ml/param/params.scala | 68 ++++- .../apache/spark/ml/param/sharedParams.scala | 28 ++- .../ml/regression/LinearRegression.scala | 96 +++++++ .../spark/ml/regression/Regressor.scala | 78 ++++++ .../apache/spark/mllib/linalg/Vectors.scala | 13 + .../apache/spark/ml/JavaPipelineSuite.java | 2 +- .../JavaLogisticRegressionSuite.java | 91 ++++++- .../regression/JavaLinearRegressionSuite.java | 89 +++++++ .../LogisticRegressionSuite.scala | 86 ++++++- .../ml/regression/LinearRegressionSuite.scala | 65 +++++ project/MimaExcludes.scala | 6 + 26 files changed, 1753 insertions(+), 156 deletions(-) create mode 100644 examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java create mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala create mode 100644 mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java create mode 100644 mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala 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 0fbee6e433608..5041e0b6d34b0 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 @@ -116,10 +116,12 @@ public static void main(String[] args) { // Make predictions on test documents. cvModel uses the best model found (lrModel). cvModel.transform(test).registerTempTable("prediction"); - DataFrame predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction"); + DataFrame predictions = jsql.sql("SELECT id, text, probability, prediction FROM prediction"); for (Row r: predictions.collect()) { - System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2) + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) + ", prediction=" + r.get(3)); } + + jsc.stop(); } } 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 new file mode 100644 index 0000000000000..42d4d7d0bef26 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java @@ -0,0 +1,217 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.util.List; + +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.ml.classification.Classifier; +import org.apache.spark.ml.classification.ClassificationModel; +import org.apache.spark.ml.param.IntParam; +import org.apache.spark.ml.param.ParamMap; +import org.apache.spark.ml.param.Params; +import org.apache.spark.ml.param.Params$; +import org.apache.spark.mllib.linalg.BLAS; +import org.apache.spark.mllib.linalg.Vector; +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.Row; +import org.apache.spark.sql.SQLContext; + + +/** + * A simple example demonstrating how to write your own learning algorithm using Estimator, + * Transformer, and other abstractions. + * This mimics {@link org.apache.spark.ml.classification.LogisticRegression}. + * + * Run with + *
+ * bin/run-example ml.JavaDeveloperApiExample
+ * 
+ */ +public class JavaDeveloperApiExample { + + public static void main(String[] args) throws Exception { + SparkConf conf = new SparkConf().setAppName("JavaDeveloperApiExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + SQLContext jsql = new SQLContext(jsc); + + // Prepare training data. + 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))); + DataFrame training = jsql.applySchema(jsc.parallelize(localTraining), LabeledPoint.class); + + // Create a LogisticRegression instance. This instance is an Estimator. + MyJavaLogisticRegression lr = new MyJavaLogisticRegression(); + // Print out the parameters, documentation, and any default values. + System.out.println("MyJavaLogisticRegression parameters:\n" + lr.explainParams() + "\n"); + + // We may set parameters using setter methods. + lr.setMaxIter(10); + + // Learn a LogisticRegression model. This uses the parameters stored in lr. + MyJavaLogisticRegressionModel model = lr.fit(training); + + // Prepare test data. + 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))); + DataFrame test = jsql.applySchema(jsc.parallelize(localTest), LabeledPoint.class); + + // Make predictions on test documents. cvModel uses the best model found (lrModel). + DataFrame results = model.transform(test); + double sumPredictions = 0; + for (Row r : results.select("features", "label", "prediction").collect()) { + sumPredictions += r.getDouble(2); + } + if (sumPredictions != 0.0) { + throw new Exception("MyJavaLogisticRegression predicted something other than 0," + + " even though all weights are 0!"); + } + + jsc.stop(); + } +} + +/** + * Example of defining a type of {@link Classifier}. + * + * NOTE: This is private since it is an example. In practice, you may not want it to be private. + */ +class MyJavaLogisticRegression + extends Classifier + implements Params { + + /** + * Param for max number of iterations + *

+ * NOTE: The usual way to add a parameter to a model or algorithm is to include: + * - val myParamName: ParamType + * - def getMyParamName + * - def setMyParamName + */ + IntParam maxIter = new IntParam(this, "maxIter", "max number of iterations"); + + int getMaxIter() { return (int)get(maxIter); } + + public MyJavaLogisticRegression() { + setMaxIter(100); + } + + // The parameter setter is in this class since it should return type MyJavaLogisticRegression. + MyJavaLogisticRegression setMaxIter(int value) { + return (MyJavaLogisticRegression)set(maxIter, value); + } + + // This method is used by fit(). + // In Java, we have to make it public since Java does not understand Scala's protected modifier. + public MyJavaLogisticRegressionModel train(DataFrame dataset, ParamMap paramMap) { + // Extract columns from data using helper method. + JavaRDD oldDataset = extractLabeledPoints(dataset, paramMap).toJavaRDD(); + + // Do learning to estimate the weight vector. + int numFeatures = oldDataset.take(1).get(0).features().size(); + Vector weights = Vectors.zeros(numFeatures); // Learning would happen here. + + // Create a model, and return it. + return new MyJavaLogisticRegressionModel(this, paramMap, weights); + } +} + +/** + * Example of defining a type of {@link ClassificationModel}. + * + * NOTE: This is private since it is an example. In practice, you may not want it to be private. + */ +class MyJavaLogisticRegressionModel + extends ClassificationModel implements Params { + + private MyJavaLogisticRegression parent_; + public MyJavaLogisticRegression parent() { return parent_; } + + private ParamMap fittingParamMap_; + public ParamMap fittingParamMap() { return fittingParamMap_; } + + private Vector weights_; + public Vector weights() { return weights_; } + + public MyJavaLogisticRegressionModel( + MyJavaLogisticRegression parent_, + ParamMap fittingParamMap_, + Vector weights_) { + this.parent_ = parent_; + this.fittingParamMap_ = fittingParamMap_; + this.weights_ = weights_; + } + + // This uses the default implementation of transform(), which reads column "features" and outputs + // columns "prediction" and "rawPrediction." + + // This uses the default implementation of predict(), which chooses the label corresponding to + // the maximum value returned by [[predictRaw()]]. + + /** + * Raw prediction for each possible label. + * The meaning of a "raw" prediction may vary between algorithms, but it intuitively gives + * a measure of confidence in each possible label (where larger = more confident). + * This internal method is used to implement [[transform()]] and output [[rawPredictionCol]]. + * + * @return vector where element i is the raw prediction for label i. + * This raw prediction may be any real number, where a larger value indicates greater + * confidence for that label. + * + * In Java, we have to make this method public since Java does not understand Scala's protected + * modifier. + */ + public Vector predictRaw(Vector features) { + double margin = BLAS.dot(features, weights_); + // There are 2 classes (binary classification), so we return a length-2 vector, + // where index i corresponds to class i (i = 0, 1). + return Vectors.dense(-margin, margin); + } + + /** + * Number of classes the label can take. 2 indicates binary classification. + */ + public int numClasses() { return 2; } + + /** + * 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()]]. + * + * In Java, we have to make this method public since Java does not understand Scala's protected + * modifier. + */ + public MyJavaLogisticRegressionModel copy() { + MyJavaLogisticRegressionModel m = + new MyJavaLogisticRegressionModel(parent_, fittingParamMap_, weights_); + Params$.MODULE$.inheritValues(this.paramMap(), this, m); + return m; + } +} 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 eaaa344be49c8..cc69e6315fdda 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 @@ -81,7 +81,7 @@ public static void main(String[] args) { // One can also combine ParamMaps. ParamMap paramMap2 = new ParamMap(); - paramMap2.put(lr.scoreCol().w("probability")); // Change 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. @@ -98,14 +98,16 @@ public static void main(String[] args) { // 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. + // 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, probability, prediction FROM results"); + jsql.sql("SELECT features, label, myProbability, prediction FROM results"); for (Row r: results.collect()) { System.out.println("(" + r.get(0) + ", " + r.get(1) + ") -> prob=" + r.get(2) + ", prediction=" + r.get(3)); } + + jsc.stop(); } } 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 82d665a3e1386..d929f1ad2014a 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 @@ -85,8 +85,10 @@ public static void main(String[] args) { model.transform(test).registerTempTable("prediction"); DataFrame 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) + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) + ", prediction=" + r.get(3)); } + + jsc.stop(); } } 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 b6c30a007d88f..a2893f78e0fec 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 @@ -23,6 +23,7 @@ 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} /** @@ -100,10 +101,10 @@ object CrossValidatorExample { // Make predictions on test documents. cvModel uses the best model found (lrModel). cvModel.transform(test) - .select("id", "text", "score", "prediction") + .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() 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 new file mode 100644 index 0000000000000..aed44238939c7 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -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. + */ + +package org.apache.spark.examples.ml + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.ml.classification.{Classifier, ClassifierParams, ClassificationModel} +import org.apache.spark.ml.param.{Params, IntParam, ParamMap} +import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.sql.{DataFrame, Row, SQLContext} + + +/** + * A simple example demonstrating how to write your own learning algorithm using Estimator, + * Transformer, and other abstractions. + * This mimics [[org.apache.spark.ml.classification.LogisticRegression]]. + * Run with + * {{{ + * bin/run-example ml.DeveloperApiExample + * }}} + */ +object DeveloperApiExample { + + def main(args: Array[String]) { + val conf = new SparkConf().setAppName("DeveloperApiExample") + val sc = new SparkContext(conf) + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + + // Prepare training data. + 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)), + LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5)))) + + // Create a LogisticRegression instance. This instance is an Estimator. + val lr = new MyLogisticRegression() + // Print out the parameters, documentation, and any default values. + println("MyLogisticRegression parameters:\n" + lr.explainParams() + "\n") + + // We may set parameters using setter methods. + lr.setMaxIter(10) + + // Learn a LogisticRegression model. This uses the parameters stored in lr. + val model = lr.fit(training) + + // 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 data. + val sumPredictions: Double = model.transform(test) + .select("features", "label", "prediction") + .collect() + .map { case Row(features: Vector, label: Double, prediction: Double) => + prediction + }.sum + assert(sumPredictions == 0.0, + "MyLogisticRegression predicted something other than 0, even though all weights are 0!") + + sc.stop() + } +} + +/** + * Example of defining a parameter trait for a user-defined type of [[Classifier]]. + * + * NOTE: This is private since it is an example. In practice, you may not want it to be private. + */ +private trait MyLogisticRegressionParams extends ClassifierParams { + + /** + * Param for max number of iterations + * + * NOTE: The usual way to add a parameter to a model or algorithm is to include: + * - val myParamName: ParamType + * - def getMyParamName + * - def setMyParamName + * Here, we have a trait to be mixed in with the Estimator and Model (MyLogisticRegression + * and MyLogisticRegressionModel). We place the setter (setMaxIter) method in the Estimator + * 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) +} + +/** + * Example of defining a type of [[Classifier]]. + * + * NOTE: This is private since it is an example. In practice, you may not want it to be private. + */ +private class MyLogisticRegression + extends Classifier[Vector, MyLogisticRegression, MyLogisticRegressionModel] + with MyLogisticRegressionParams { + + setMaxIter(100) // Initialize + + // The parameter setter is in this class since it should return type MyLogisticRegression. + def setMaxIter(value: Int): this.type = set(maxIter, value) + + // This method is used by fit() + override protected def train( + dataset: DataFrame, + paramMap: ParamMap): MyLogisticRegressionModel = { + // Extract columns from data using helper method. + val oldDataset = extractLabeledPoints(dataset, paramMap) + + // Do learning to estimate the weight vector. + val numFeatures = oldDataset.take(1)(0).features.size + val weights = Vectors.zeros(numFeatures) // Learning would happen here. + + // Create a model, and return it. + new MyLogisticRegressionModel(this, paramMap, weights) + } +} + +/** + * Example of defining a type of [[ClassificationModel]]. + * + * NOTE: This is private since it is an example. In practice, you may not want it to be private. + */ +private class MyLogisticRegressionModel( + override val parent: MyLogisticRegression, + override val fittingParamMap: ParamMap, + val weights: Vector) + extends ClassificationModel[Vector, MyLogisticRegressionModel] + with MyLogisticRegressionParams { + + // This uses the default implementation of transform(), which reads column "features" and outputs + // columns "prediction" and "rawPrediction." + + // This uses the default implementation of predict(), which chooses the label corresponding to + // the maximum value returned by [[predictRaw()]]. + + /** + * Raw prediction for each possible label. + * The meaning of a "raw" prediction may vary between algorithms, but it intuitively gives + * a measure of confidence in each possible label (where larger = more confident). + * This internal method is used to implement [[transform()]] and output [[rawPredictionCol]]. + * + * @return vector where element i is the raw prediction for label i. + * This raw prediction may be any real number, where a larger value indicates greater + * confidence for that label. + */ + override protected def predictRaw(features: Vector): Vector = { + val margin = BLAS.dot(features, weights) + // There are 2 classes (binary classification), so we return a length-2 vector, + // where index i corresponds to class i (i = 0, 1). + Vectors.dense(-margin, margin) + } + + /** Number of classes the label can take. 2 indicates binary classification. */ + override val numClasses: Int = 2 + + /** + * 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()]]. + */ + override protected def copy(): MyLogisticRegressionModel = { + val m = new MyLogisticRegressionModel(parent, fittingParamMap, weights) + Params.inheritValues(this.paramMap, this, m) + m + } +} 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 4d1530cd1349f..80c9f5ff5781e 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 @@ -72,7 +72,7 @@ object SimpleParamsExample { paramMap.put(lr.regParam -> 0.1, lr.threshold -> 0.55) // Specify multiple Params. // One can also combine ParamMaps. - val paramMap2 = ParamMap(lr.scoreCol -> "probability") // Change 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. @@ -80,21 +80,21 @@ object SimpleParamsExample { val model2 = lr.fit(training, paramMapCombined) println("Model 2 was fit using parameters: " + model2.fittingParamMap) - // Prepare test documents. + // 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. + // 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) - .select("features", "label", "probability", "prediction") + .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() 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 dbbe01dd5ce8e..968cb292120d8 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 @@ -23,6 +23,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} @BeanInfo @@ -79,10 +80,10 @@ object SimpleTextClassificationPipeline { // Make predictions on test documents. model.transform(test) - .select("id", "text", "score", "prediction") + .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() 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 bc3defe968afd..eff7ef925dfbd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala @@ -34,7 +34,8 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage with Params { * Fits a single model to the input data with optional parameters. * * @param dataset input dataset - * @param paramPairs optional list of param pairs (overwrite embedded params) + * @param paramPairs Optional list of param pairs. + * These values override any specified in this Estimator's embedded ParamMap. * @return fitted model */ @varargs @@ -47,7 +48,8 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage with Params { * Fits a single model to the input data with provided parameter map. * * @param dataset input dataset - * @param paramMap parameter map + * @param paramMap Parameter map. + * These values override any specified in this Estimator's embedded ParamMap. * @return fitted model */ def fit(dataset: DataFrame, paramMap: ParamMap): M @@ -58,7 +60,8 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage with Params { * Subclasses could overwrite this to optimize multi-model training. * * @param dataset input dataset - * @param paramMaps an array of parameter maps + * @param paramMaps An array of parameter maps. + * These values override any specified in this Estimator's embedded ParamMap. * @return fitted models, matching the input parameter maps */ def fit(dataset: DataFrame, paramMaps: Array[ParamMap]): Seq[M] = { 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 new file mode 100644 index 0000000000000..1bf8eb4640d11 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.{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.DataFrame +import org.apache.spark.sql.types.{DataType, DoubleType, StructType} + + +/** + * :: DeveloperApi :: + * Params for classification. + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. + */ +@DeveloperApi +private[spark] trait ClassifierParams extends PredictorParams + with HasRawPredictionCol { + + override protected def validateAndTransformSchema( + schema: StructType, + paramMap: ParamMap, + fitting: Boolean, + featuresDataType: DataType): StructType = { + val parentSchema = super.validateAndTransformSchema(schema, paramMap, fitting, featuresDataType) + val map = this.paramMap ++ paramMap + addOutputColumn(parentSchema, map(rawPredictionCol), new VectorUDT) + } +} + +/** + * :: AlphaComponent :: + * Single-label binary or multiclass classification. + * Classes are indexed {0, 1, ..., numClasses - 1}. + * + * @tparam FeaturesType Type of input features. E.g., [[Vector]] + * @tparam E Concrete Estimator type + * @tparam M Concrete Model type + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. + */ +@AlphaComponent +private[spark] abstract class Classifier[ + FeaturesType, + E <: Classifier[FeaturesType, E, M], + M <: ClassificationModel[FeaturesType, M]] + extends Predictor[FeaturesType, E, M] + with ClassifierParams { + + def setRawPredictionCol(value: String): E = + set(rawPredictionCol, value).asInstanceOf[E] + + // TODO: defaultEvaluator (follow-up PR) +} + +/** + * :: AlphaComponent :: + * Model produced by a [[Classifier]]. + * Classes are indexed {0, 1, ..., numClasses - 1}. + * + * @tparam FeaturesType Type of input features. E.g., [[Vector]] + * @tparam M Concrete Model type + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. + */ +@AlphaComponent +private[spark] +abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[FeaturesType, M]] + extends PredictionModel[FeaturesType, M] with ClassifierParams { + + def setRawPredictionCol(value: String): M = set(rawPredictionCol, value).asInstanceOf[M] + + /** Number of classes (values which the label can take). */ + def numClasses: Int + + /** + * Transforms dataset by reading from [[featuresCol]], and appending new columns as specified by + * parameters: + * - predicted labels as [[predictionCol]] of type [[Double]] + * - raw predictions (confidences) as [[rawPredictionCol]] of type [[Vector]]. + * + * @param dataset input dataset + * @param paramMap additional parameters, overwrite embedded params + * @return transformed dataset + */ + override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { + // This default implementation should be overridden as needed. + + // Check schema + transformSchema(dataset.schema, paramMap, logging = true) + val map = this.paramMap ++ paramMap + + // Prepare model + val tmpModel = if (paramMap.size != 0) { + val tmpModel = this.copy() + Params.inheritValues(paramMap, parent, tmpModel) + tmpModel + } else { + this + } + + val (numColsOutput, outputData) = + ClassificationModel.transformColumnsImpl[FeaturesType](dataset, tmpModel, map) + if (numColsOutput == 0) { + logWarning(s"$uid: ClassificationModel.transform() was called as NOOP" + + " since no output columns were set.") + } + outputData + } + + /** + * :: DeveloperApi :: + * + * Predict label for the given features. + * This internal method is used to implement [[transform()]] and output [[predictionCol]]. + * + * This default implementation for classification predicts the index of the maximum value + * from [[predictRaw()]]. + */ + @DeveloperApi + override protected def predict(features: FeaturesType): Double = { + predictRaw(features).toArray.zipWithIndex.maxBy(_._1)._2 + } + + /** + * :: DeveloperApi :: + * + * Raw prediction for each possible label. + * The meaning of a "raw" prediction may vary between algorithms, but it intuitively gives + * a measure of confidence in each possible label (where larger = more confident). + * This internal method is used to implement [[transform()]] and output [[rawPredictionCol]]. + * + * @return vector where element i is the raw prediction for label i. + * This raw prediction may be any real number, where a larger value indicates greater + * confidence for that label. + */ + @DeveloperApi + protected def predictRaw(features: FeaturesType): Vector + +} + +private[ml] object ClassificationModel { + + /** + * Added prediction column(s). This is separated from [[ClassificationModel.transform()]] + * since it is used by [[org.apache.spark.ml.classification.ProbabilisticClassificationModel]]. + * @param dataset Input dataset + * @param map Parameter map. This will NOT be merged with the embedded paramMap; the merge + * should already be done. + * @return (number of columns added, transformed dataset) + */ + def transformColumnsImpl[FeaturesType]( + dataset: DataFrame, + model: ClassificationModel[FeaturesType, _], + map: ParamMap): (Int, DataFrame) = { + + // Output selected columns only. + // This is a bit complicated since it tries to avoid repeated computation. + var tmpData = dataset + var numColsOutput = 0 + 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))) + 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))) + 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))) + 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 b46a5cd8bdf29..c146fe244c66e 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 @@ -18,61 +18,32 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml._ import org.apache.spark.ml.param._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS -import org.apache.spark.mllib.linalg.{BLAS, Vector, VectorUDT} -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.sql._ +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.types.{DoubleType, StructField, StructType} +import org.apache.spark.sql.types.DoubleType import org.apache.spark.storage.StorageLevel + /** - * :: AlphaComponent :: * Params for logistic regression. */ -@AlphaComponent -private[classification] trait LogisticRegressionParams extends Params - with HasRegParam with HasMaxIter with HasLabelCol with HasThreshold with HasFeaturesCol - with HasScoreCol with HasPredictionCol { +private[classification] trait LogisticRegressionParams extends ProbabilisticClassifierParams + with HasRegParam with HasMaxIter with HasThreshold - /** - * Validates and transforms the input schema with the provided param map. - * @param schema input schema - * @param paramMap additional parameters - * @param fitting whether this is in fitting - * @return output schema - */ - protected def validateAndTransformSchema( - schema: StructType, - paramMap: ParamMap, - fitting: Boolean): StructType = { - val map = this.paramMap ++ paramMap - val featuresType = schema(map(featuresCol)).dataType - // TODO: Support casting Array[Double] and Array[Float] to Vector. - require(featuresType.isInstanceOf[VectorUDT], - s"Features column ${map(featuresCol)} must be a vector column but got $featuresType.") - if (fitting) { - val labelType = schema(map(labelCol)).dataType - require(labelType == DoubleType, - s"Cannot convert label column ${map(labelCol)} of type $labelType to a double column.") - } - val fieldNames = schema.fieldNames - require(!fieldNames.contains(map(scoreCol)), s"Score column ${map(scoreCol)} already exists.") - require(!fieldNames.contains(map(predictionCol)), - s"Prediction column ${map(predictionCol)} already exists.") - val outputFields = schema.fields ++ Seq( - StructField(map(scoreCol), DoubleType, false), - StructField(map(predictionCol), DoubleType, false)) - StructType(outputFields) - } -} /** + * :: AlphaComponent :: + * * Logistic regression. + * Currently, this class only supports binary classification. */ -class LogisticRegression extends Estimator[LogisticRegressionModel] with LogisticRegressionParams { +@AlphaComponent +class LogisticRegression + extends ProbabilisticClassifier[Vector, LogisticRegression, LogisticRegressionModel] + with LogisticRegressionParams { setRegParam(0.1) setMaxIter(100) @@ -80,68 +51,151 @@ class LogisticRegression extends Estimator[LogisticRegressionModel] with Logisti def setRegParam(value: Double): this.type = set(regParam, value) def setMaxIter(value: Int): this.type = set(maxIter, value) - def setLabelCol(value: String): this.type = set(labelCol, value) def setThreshold(value: Double): this.type = set(threshold, value) - def setFeaturesCol(value: String): this.type = set(featuresCol, value) - def setScoreCol(value: String): this.type = set(scoreCol, value) - def setPredictionCol(value: String): this.type = set(predictionCol, value) - override def fit(dataset: DataFrame, paramMap: ParamMap): LogisticRegressionModel = { - transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap - val instances = dataset.select(map(labelCol), map(featuresCol)) - .map { case Row(label: Double, features: Vector) => - LabeledPoint(label, features) - }.persist(StorageLevel.MEMORY_AND_DISK) + override protected def train(dataset: DataFrame, paramMap: ParamMap): LogisticRegressionModel = { + // Extract columns from data. If dataset is persisted, do not persist oldDataset. + val oldDataset = extractLabeledPoints(dataset, paramMap) + val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE + if (handlePersistence) { + oldDataset.persist(StorageLevel.MEMORY_AND_DISK) + } + + // Train model val lr = new LogisticRegressionWithLBFGS lr.optimizer - .setRegParam(map(regParam)) - .setNumIterations(map(maxIter)) - val lrm = new LogisticRegressionModel(this, map, lr.run(instances).weights) - instances.unpersist() - // copy model params - Params.inheritValues(map, this, lrm) - lrm - } + .setRegParam(paramMap(regParam)) + .setNumIterations(paramMap(maxIter)) + val oldModel = lr.run(oldDataset) + val lrm = new LogisticRegressionModel(this, paramMap, oldModel.weights, oldModel.intercept) - private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - validateAndTransformSchema(schema, paramMap, fitting = true) + if (handlePersistence) { + oldDataset.unpersist() + } + lrm } } + /** * :: AlphaComponent :: + * * Model produced by [[LogisticRegression]]. */ @AlphaComponent class LogisticRegressionModel private[ml] ( override val parent: LogisticRegression, override val fittingParamMap: ParamMap, - weights: Vector) - extends Model[LogisticRegressionModel] with LogisticRegressionParams { + val weights: Vector, + val intercept: Double) + extends ProbabilisticClassificationModel[Vector, LogisticRegressionModel] + with LogisticRegressionParams { + + setThreshold(0.5) def setThreshold(value: Double): this.type = set(threshold, value) - def setFeaturesCol(value: String): this.type = set(featuresCol, value) - def setScoreCol(value: String): this.type = set(scoreCol, value) - def setPredictionCol(value: String): this.type = set(predictionCol, value) - private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - validateAndTransformSchema(schema, paramMap, fitting = false) + private val margin: Vector => Double = (features) => { + BLAS.dot(features, weights) + intercept + } + + private val score: Vector => Double = (features) => { + val m = margin(features) + 1.0 / (1.0 + math.exp(-m)) } override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { + // This is overridden (a) to be more efficient (avoiding re-computing values when creating + // multiple output columns) and (b) to handle threshold, which the abstractions do not use. + // TODO: We should abstract away the steps defined by UDFs below so that the abstractions + // can call whichever UDFs are needed to create the output columns. + + // Check schema transformSchema(dataset.schema, paramMap, logging = true) + val map = this.paramMap ++ paramMap - val scoreFunction = udf { v: Vector => - val margin = BLAS.dot(v, weights) - 1.0 / (1.0 + math.exp(-margin)) + + // Output selected columns only. + // This is a bit complicated since it tries to avoid repeated computation. + // rawPrediction (-margin, margin) + // probability (1.0-score, score) + // prediction (max margin) + var tmpData = dataset + 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))) + numColsOutput += 1 + } + if (map(probabilityCol) != "") { + if (map(rawPredictionCol) != "") { + val raw2prob: Vector => Vector = { (rawPreds: Vector) => + val prob1 = 1.0 / (1.0 + math.exp(-rawPreds(1))) + Vectors.dense(1.0 - prob1, prob1) + } + tmpData = tmpData.select($"*", + callUDF(raw2prob, new VectorUDT, col(map(rawPredictionCol))).as(map(probabilityCol))) + } else { + val features2prob: Vector => Vector = (features: Vector) => predictProbabilities(features) + tmpData = tmpData.select($"*", + callUDF(features2prob, new VectorUDT, col(map(featuresCol))).as(map(probabilityCol))) + } + numColsOutput += 1 } - val t = map(threshold) - val predictFunction = udf { score: Double => - if (score > t) 1.0 else 0.0 + if (map(predictionCol) != "") { + val t = map(threshold) + if (map(probabilityCol) != "") { + val predict: Vector => Double = { probs: Vector => + if (probs(1) > t) 1.0 else 0.0 + } + tmpData = tmpData.select($"*", + callUDF(predict, DoubleType, col(map(probabilityCol))).as(map(predictionCol))) + } else if (map(rawPredictionCol) != "") { + val predict: Vector => Double = { 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))) + } else { + val predict: Vector => Double = (features: Vector) => this.predict(features) + tmpData = tmpData.select($"*", + callUDF(predict, DoubleType, col(map(featuresCol))).as(map(predictionCol))) + } + numColsOutput += 1 } - dataset - .select($"*", scoreFunction(col(map(featuresCol))).as(map(scoreCol))) - .select($"*", predictFunction(col(map(scoreCol))).as(map(predictionCol))) + if (numColsOutput == 0) { + this.logWarning(s"$uid: LogisticRegressionModel.transform() was called as NOOP" + + " since no output columns were set.") + } + tmpData + } + + override val numClasses: Int = 2 + + /** + * Predict label for the given feature vector. + * 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 + } + + override protected def predictProbabilities(features: Vector): Vector = { + val s = score(features) + Vectors.dense(1.0 - s, s) + } + + override protected def predictRaw(features: Vector): Vector = { + val m = margin(features) + Vectors.dense(0.0, m) + } + + override protected def copy(): LogisticRegressionModel = { + val m = new LogisticRegressionModel(parent, fittingParamMap, weights, intercept) + Params.inheritValues(this.paramMap, 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 new file mode 100644 index 0000000000000..1202528ca654e --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.classification + +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.types.{DataType, StructType} + + +/** + * Params for probabilistic classification. + */ +private[classification] trait ProbabilisticClassifierParams + extends ClassifierParams with HasProbabilityCol { + + override protected def validateAndTransformSchema( + schema: StructType, + paramMap: ParamMap, + fitting: Boolean, + featuresDataType: DataType): StructType = { + val parentSchema = super.validateAndTransformSchema(schema, paramMap, fitting, featuresDataType) + val map = this.paramMap ++ paramMap + addOutputColumn(parentSchema, map(probabilityCol), new VectorUDT) + } +} + + +/** + * :: AlphaComponent :: + * + * Single-label binary or multiclass classifier which can output class conditional probabilities. + * + * @tparam FeaturesType Type of input features. E.g., [[Vector]] + * @tparam E Concrete Estimator type + * @tparam M Concrete Model type + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. + */ +@AlphaComponent +private[spark] abstract class ProbabilisticClassifier[ + FeaturesType, + E <: ProbabilisticClassifier[FeaturesType, E, M], + M <: ProbabilisticClassificationModel[FeaturesType, M]] + extends Classifier[FeaturesType, E, M] with ProbabilisticClassifierParams { + + def setProbabilityCol(value: String): E = set(probabilityCol, value).asInstanceOf[E] +} + + +/** + * :: AlphaComponent :: + * + * Model produced by a [[ProbabilisticClassifier]]. + * Classes are indexed {0, 1, ..., numClasses - 1}. + * + * @tparam FeaturesType Type of input features. E.g., [[Vector]] + * @tparam M Concrete Model type + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. + */ +@AlphaComponent +private[spark] abstract class ProbabilisticClassificationModel[ + FeaturesType, + M <: ProbabilisticClassificationModel[FeaturesType, M]] + extends ClassificationModel[FeaturesType, M] with ProbabilisticClassifierParams { + + def setProbabilityCol(value: String): M = set(probabilityCol, value).asInstanceOf[M] + + /** + * Transforms dataset by reading from [[featuresCol]], and appending new columns as specified by + * parameters: + * - predicted labels as [[predictionCol]] of type [[Double]] + * - raw predictions (confidences) as [[rawPredictionCol]] of type [[Vector]] + * - probability of each class as [[probabilityCol]] of type [[Vector]]. + * + * @param dataset input dataset + * @param paramMap additional parameters, overwrite embedded params + * @return transformed dataset + */ + override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { + // This default implementation should be overridden as needed. + + // Check schema + transformSchema(dataset.schema, paramMap, logging = true) + val map = this.paramMap ++ paramMap + + // Prepare model + val tmpModel = if (paramMap.size != 0) { + val tmpModel = this.copy() + Params.inheritValues(paramMap, parent, tmpModel) + tmpModel + } else { + this + } + + val (numColsOutput, outputData) = + ClassificationModel.transformColumnsImpl[FeaturesType](dataset, tmpModel, map) + + // Output selected columns only. + if (map(probabilityCol) != "") { + // output probabilities + val features2probs: FeaturesType => Vector = (features) => { + tmpModel.predictProbabilities(features) + } + outputData.select($"*", + callUDF(features2probs, new VectorUDT, col(map(featuresCol))).as(map(probabilityCol))) + } else { + if (numColsOutput == 0) { + this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() was called as NOOP" + + " since no output columns were set.") + } + outputData + } + } + + /** + * :: DeveloperApi :: + * + * Predict the probability of each class given the features. + * These predictions are also called class conditional probabilities. + * + * WARNING: Not all models output well-calibrated probability estimates! These probabilities + * should be treated as confidences, not precise probabilities. + * + * This internal method is used to implement [[transform()]] and output [[probabilityCol]]. + */ + @DeveloperApi + protected def predictProbabilities(features: FeaturesType): Vector +} 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 1979ab9eb6516..f21a30627e540 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 @@ -18,19 +18,22 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml._ +import org.apache.spark.ml.Evaluator import org.apache.spark.ml.param._ 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 :: + * * Evaluator for binary classification, which expects two input columns: score and label. */ @AlphaComponent class BinaryClassificationEvaluator extends Evaluator with Params - with HasScoreCol with HasLabelCol { + with HasRawPredictionCol with HasLabelCol { /** param for metric name in evaluation */ val metricName: Param[String] = new Param(this, "metricName", @@ -38,23 +41,20 @@ class BinaryClassificationEvaluator extends Evaluator with Params def getMetricName: String = get(metricName) def setMetricName(value: String): this.type = set(metricName, value) - def setScoreCol(value: String): this.type = set(scoreCol, value) + def setScoreCol(value: String): this.type = set(rawPredictionCol, value) def setLabelCol(value: String): this.type = set(labelCol, value) override def evaluate(dataset: DataFrame, paramMap: ParamMap): Double = { val map = this.paramMap ++ paramMap val schema = dataset.schema - val scoreType = schema(map(scoreCol)).dataType - require(scoreType == DoubleType, - s"Score column ${map(scoreCol)} must be double type but found $scoreType") - val labelType = schema(map(labelCol)).dataType - require(labelType == DoubleType, - s"Label column ${map(labelCol)} must be double type but found $labelType") + checkInputColumn(schema, map(rawPredictionCol), new VectorUDT) + checkInputColumn(schema, map(labelCol), DoubleType) - val scoreAndLabels = dataset.select(map(scoreCol), map(labelCol)) - .map { case Row(score: Double, label: Double) => - (score, label) + // TODO: When dataset metadata has been implemented, check rawPredictionCol vector length = 2. + val scoreAndLabels = dataset.select(map(rawPredictionCol), map(labelCol)) + .map { case Row(rawPrediction: Vector, label: Double) => + (rawPrediction(1), label) } val metrics = new BinaryClassificationMetrics(scoreAndLabels) val metric = map(metricName) match { 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 e622a5cf9e6f3..0b1f90daa7d8e 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 @@ -29,11 +29,11 @@ import org.apache.spark.sql.types.{DataType, StringType, ArrayType} @AlphaComponent class Tokenizer extends UnaryTransformer[String, Seq[String], Tokenizer] { - protected override def createTransformFunc(paramMap: ParamMap): String => Seq[String] = { + override protected def createTransformFunc(paramMap: ParamMap): String => Seq[String] = { _.toLowerCase.split("\\s") } - protected override def validateInputType(inputType: DataType): Unit = { + override protected def validateInputType(inputType: DataType): Unit = { require(inputType == StringType, s"Input type must be string type but got $inputType.") } 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 new file mode 100644 index 0000000000000..89b53f3890ea3 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -0,0 +1,234 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.estimator + +import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.param._ +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.types.{DataType, DoubleType, StructType} + + +/** + * :: DeveloperApi :: + * + * Trait for parameters for prediction (regression and classification). + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. + */ +@DeveloperApi +private[spark] trait PredictorParams extends Params + with HasLabelCol with HasFeaturesCol with HasPredictionCol { + + /** + * Validates and transforms the input schema with the provided param map. + * @param schema input schema + * @param paramMap additional parameters + * @param fitting whether this is in fitting + * @param featuresDataType SQL DataType for FeaturesType. + * E.g., [[org.apache.spark.mllib.linalg.VectorUDT]] for vector features. + * @return output schema + */ + protected def validateAndTransformSchema( + schema: StructType, + paramMap: ParamMap, + fitting: Boolean, + featuresDataType: DataType): StructType = { + val map = this.paramMap ++ paramMap + // TODO: Support casting Array[Double] and Array[Float] to Vector when FeaturesType = Vector + checkInputColumn(schema, map(featuresCol), featuresDataType) + if (fitting) { + // TODO: Allow other numeric types + checkInputColumn(schema, map(labelCol), DoubleType) + } + addOutputColumn(schema, map(predictionCol), DoubleType) + } +} + +/** + * :: AlphaComponent :: + * + * Abstraction for prediction problems (regression and classification). + * + * @tparam FeaturesType Type of features. + * E.g., [[org.apache.spark.mllib.linalg.VectorUDT]] for vector features. + * @tparam Learner Specialization of this class. If you subclass this type, use this type + * parameter to specify the concrete type. + * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type + * parameter to specify the concrete type for the corresponding model. + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. + */ +@AlphaComponent +private[spark] abstract class Predictor[ + FeaturesType, + Learner <: Predictor[FeaturesType, Learner, M], + M <: PredictionModel[FeaturesType, M]] + extends Estimator[M] with PredictorParams { + + def setLabelCol(value: String): Learner = set(labelCol, value).asInstanceOf[Learner] + def setFeaturesCol(value: String): Learner = set(featuresCol, value).asInstanceOf[Learner] + def setPredictionCol(value: String): Learner = set(predictionCol, value).asInstanceOf[Learner] + + override def fit(dataset: DataFrame, paramMap: ParamMap): M = { + // 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 model = train(dataset, map) + Params.inheritValues(map, this, model) // copy params to model + model + } + + /** + * :: DeveloperApi :: + * + * Train a model using the given dataset and parameters. + * Developers can implement this instead of [[fit()]] to avoid dealing with schema validation + * and copying parameters into the model. + * + * @param dataset Training dataset + * @param paramMap Parameter map. Unlike [[fit()]]'s paramMap, this paramMap has already + * been combined with the embedded ParamMap. + * @return Fitted model + */ + @DeveloperApi + protected def train(dataset: DataFrame, paramMap: ParamMap): M + + /** + * :: DeveloperApi :: + * + * Returns the SQL DataType corresponding to the FeaturesType type parameter. + * + * This is used by [[validateAndTransformSchema()]]. + * This workaround is needed since SQL has different APIs for Scala and Java. + * + * The default value is VectorUDT, but it may be overridden if FeaturesType is not Vector. + */ + @DeveloperApi + protected def featuresDataType: DataType = new VectorUDT + + private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + validateAndTransformSchema(schema, paramMap, fitting = true, featuresDataType) + } + + /** + * Extract [[labelCol]] and [[featuresCol]] from the given dataset, + * and put it in an RDD with strong types. + */ + protected def extractLabeledPoints(dataset: DataFrame, paramMap: ParamMap): RDD[LabeledPoint] = { + val map = this.paramMap ++ paramMap + dataset.select(map(labelCol), map(featuresCol)) + .map { case Row(label: Double, features: Vector) => + LabeledPoint(label, features) + } + } +} + +/** + * :: AlphaComponent :: + * + * Abstraction for a model for prediction tasks (regression and classification). + * + * @tparam FeaturesType Type of features. + * E.g., [[org.apache.spark.mllib.linalg.VectorUDT]] for vector features. + * @tparam M Specialization of [[PredictionModel]]. If you subclass this type, use this type + * parameter to specify the concrete type for the corresponding model. + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. + */ +@AlphaComponent +private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, M]] + extends Model[M] with PredictorParams { + + def setFeaturesCol(value: String): M = set(featuresCol, value).asInstanceOf[M] + + def setPredictionCol(value: String): M = set(predictionCol, value).asInstanceOf[M] + + /** + * :: DeveloperApi :: + * + * Returns the SQL DataType corresponding to the FeaturesType type parameter. + * + * This is used by [[validateAndTransformSchema()]]. + * This workaround is needed since SQL has different APIs for Scala and Java. + * + * The default value is VectorUDT, but it may be overridden if FeaturesType is not Vector. + */ + @DeveloperApi + protected def featuresDataType: DataType = new VectorUDT + + private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + validateAndTransformSchema(schema, paramMap, fitting = false, featuresDataType) + } + + /** + * Transforms dataset by reading from [[featuresCol]], calling [[predict()]], and storing + * the predictions as a new column [[predictionCol]]. + * + * @param dataset input dataset + * @param paramMap additional parameters, overwrite embedded params + * @return transformed dataset with [[predictionCol]] of type [[Double]] + */ + override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { + // This default implementation should be overridden as needed. + + // Check schema + transformSchema(dataset.schema, paramMap, logging = true) + val map = this.paramMap ++ paramMap + + // Prepare model + val tmpModel = if (paramMap.size != 0) { + val tmpModel = this.copy() + Params.inheritValues(paramMap, parent, tmpModel) + tmpModel + } else { + this + } + + if (map(predictionCol) != "") { + val pred: FeaturesType => Double = (features) => { + tmpModel.predict(features) + } + dataset.select($"*", callUDF(pred, DoubleType, col(map(featuresCol))).as(map(predictionCol))) + } else { + this.logWarning(s"$uid: Predictor.transform() was called as NOOP" + + " since no output columns were set.") + dataset + } + } + + /** + * :: DeveloperApi :: + * + * Predict label for the given features. + * This internal method is used to implement [[transform()]] and output [[predictionCol]]. + */ + @DeveloperApi + protected def predict(features: FeaturesType): Double + + /** + * Create a copy of the model. + * The copy is shallow, except for the embedded paramMap, which gets a deep copy. + */ + protected def copy(): M +} 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 5fb4379e23c2f..17ece897a6c55 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 @@ -22,8 +22,10 @@ import scala.collection.mutable import java.lang.reflect.Modifier -import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} import org.apache.spark.ml.Identifiable +import org.apache.spark.sql.types.{DataType, StructField, StructType} + /** * :: AlphaComponent :: @@ -65,37 +67,47 @@ class Param[T] ( // 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] = None) +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) + 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] = None) +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) + 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] = None) +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) + 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] = None) +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) + 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] = None) +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) + override def w(value: Boolean): ParamPair[Boolean] = super.w(value) } @@ -158,7 +170,7 @@ trait Params extends Identifiable with Serializable { /** * Sets a parameter in the embedded param map. */ - private[ml] def set[T](param: Param[T], value: T): this.type = { + protected def set[T](param: Param[T], value: T): this.type = { require(param.parent.eq(this)) paramMap.put(param.asInstanceOf[Param[Any]], value) this @@ -174,7 +186,7 @@ trait Params extends Identifiable with Serializable { /** * Gets the value of a parameter in the embedded param map. */ - private[ml] def get[T](param: Param[T]): T = { + protected def get[T](param: Param[T]): T = { require(param.parent.eq(this)) paramMap(param) } @@ -183,9 +195,40 @@ trait Params extends Identifiable with Serializable { * Internal param map. */ protected val paramMap: ParamMap = ParamMap.empty + + /** + * 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. + */ + 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)}") + } + + 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.") + val outputFields = schema.fields ++ Seq(StructField(colName, dataType, nullable = false)) + StructType(outputFields) + } } -private[ml] object Params { +/** + * :: DeveloperApi :: + * + * Helper functionality for developers. + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. + */ +@DeveloperApi +private[spark] object Params { /** * Copies parameter values from the parent estimator to the child model it produced. @@ -279,7 +322,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten def copy: ParamMap = new ParamMap(map.clone()) override def toString: String = { - map.map { case (param, value) => + map.toSeq.sortBy(_._1.name).map { case (param, value) => s"\t${param.parent.uid}-${param.name}: $value" }.mkString("{\n", ",\n", "\n}") } @@ -310,6 +353,11 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten ParamPair(param, value) } } + + /** + * Number of param pairs in this set. + */ + def size: Int = map.size } object ParamMap { 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 ef141d3eb2b06..32fc74462ef4a 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 @@ -17,6 +17,12 @@ 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 */ val regParam: DoubleParam = new DoubleParam(this, "regParam", "regularization parameter") @@ -42,12 +48,6 @@ private[ml] trait HasLabelCol extends Params { def getLabelCol: String = get(labelCol) } -private[ml] trait HasScoreCol extends Params { - /** param for score column name */ - val scoreCol: Param[String] = new Param(this, "scoreCol", "score column name", Some("score")) - def getScoreCol: String = get(scoreCol) -} - private[ml] trait HasPredictionCol extends Params { /** param for prediction column name */ val predictionCol: Param[String] = @@ -55,6 +55,22 @@ private[ml] trait HasPredictionCol extends Params { def getPredictionCol: String = get(predictionCol) } +private[ml] trait HasRawPredictionCol extends Params { + /** param for raw prediction column name */ + val rawPredictionCol: Param[String] = + new Param(this, "rawPredictionCol", "raw prediction (a.k.a. confidence) column name", + Some("rawPrediction")) + def getRawPredictionCol: String = get(rawPredictionCol) +} + +private[ml] trait HasProbabilityCol extends Params { + /** param for predicted class conditional probabilities column name */ + val probabilityCol: Param[String] = + new Param(this, "probabilityCol", "column name for predicted class conditional probabilities", + Some("probability")) + def getProbabilityCol: String = get(probabilityCol) +} + private[ml] trait HasThreshold extends Params { /** param for threshold in (binary) prediction */ val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold in prediction") 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 new file mode 100644 index 0000000000000..d5a7bdafcb623 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -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. + */ + +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.mllib.linalg.{BLAS, Vector} +import org.apache.spark.mllib.regression.LinearRegressionWithSGD +import org.apache.spark.sql.DataFrame +import org.apache.spark.storage.StorageLevel + + +/** + * Params for linear regression. + */ +private[regression] trait LinearRegressionParams extends RegressorParams + with HasRegParam with HasMaxIter + + +/** + * :: AlphaComponent :: + * + * Linear regression. + */ +@AlphaComponent +class LinearRegression extends Regressor[Vector, LinearRegression, LinearRegressionModel] + with LinearRegressionParams { + + setRegParam(0.1) + setMaxIter(100) + + def setRegParam(value: Double): this.type = set(regParam, value) + def setMaxIter(value: Int): this.type = set(maxIter, value) + + override protected def train(dataset: DataFrame, paramMap: ParamMap): LinearRegressionModel = { + // Extract columns from data. If dataset is persisted, do not persist oldDataset. + val oldDataset = extractLabeledPoints(dataset, paramMap) + val handlePersistence = dataset.rdd.getStorageLevel == StorageLevel.NONE + if (handlePersistence) { + oldDataset.persist(StorageLevel.MEMORY_AND_DISK) + } + + // Train model + val lr = new LinearRegressionWithSGD() + lr.optimizer + .setRegParam(paramMap(regParam)) + .setNumIterations(paramMap(maxIter)) + val model = lr.run(oldDataset) + val lrm = new LinearRegressionModel(this, paramMap, model.weights, model.intercept) + + if (handlePersistence) { + oldDataset.unpersist() + } + lrm + } +} + +/** + * :: AlphaComponent :: + * + * Model produced by [[LinearRegression]]. + */ +@AlphaComponent +class LinearRegressionModel private[ml] ( + override val parent: LinearRegression, + override val fittingParamMap: ParamMap, + val weights: Vector, + val intercept: Double) + extends RegressionModel[Vector, LinearRegressionModel] + with LinearRegressionParams { + + override protected def predict(features: Vector): Double = { + BLAS.dot(features, weights) + intercept + } + + override protected def copy(): LinearRegressionModel = { + val m = new LinearRegressionModel(parent, fittingParamMap, weights, intercept) + Params.inheritValues(this.paramMap, this, m) + m + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala new file mode 100644 index 0000000000000..d679085eeafe1 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/Regressor.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.{DeveloperApi, AlphaComponent} +import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} + +/** + * :: DeveloperApi :: + * Params for regression. + * Currently empty, but may add functionality later. + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. + */ +@DeveloperApi +private[spark] trait RegressorParams extends PredictorParams + +/** + * :: AlphaComponent :: + * + * Single-label regression + * + * @tparam FeaturesType Type of input features. E.g., [[org.apache.spark.mllib.linalg.Vector]] + * @tparam Learner Concrete Estimator type + * @tparam M Concrete Model type + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. + */ +@AlphaComponent +private[spark] abstract class Regressor[ + FeaturesType, + Learner <: Regressor[FeaturesType, Learner, M], + M <: RegressionModel[FeaturesType, M]] + extends Predictor[FeaturesType, Learner, M] + with RegressorParams { + + // TODO: defaultEvaluator (follow-up PR) +} + +/** + * :: AlphaComponent :: + * + * Model produced by a [[Regressor]]. + * + * @tparam FeaturesType Type of input features. E.g., [[org.apache.spark.mllib.linalg.Vector]] + * @tparam M Concrete Model type. + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. + */ +@AlphaComponent +private[spark] abstract class RegressionModel[FeaturesType, M <: RegressionModel[FeaturesType, M]] + extends PredictionModel[FeaturesType, M] with RegressorParams { + + /** + * :: DeveloperApi :: + * + * Predict real-valued label for the given features. + * This internal method is used to implement [[transform()]] and output [[predictionCol]]. + */ + @DeveloperApi + protected def predict(features: FeaturesType): Double + +} 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 77785bdbd03d9..480bbfb5fe94a 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 @@ -26,6 +26,7 @@ import scala.collection.JavaConverters._ import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV} import org.apache.spark.SparkException +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.util.NumericParser import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow @@ -110,9 +111,14 @@ sealed trait Vector extends Serializable { } /** + * :: DeveloperApi :: + * * User-defined type for [[Vector]] which allows easy interaction with SQL * via [[org.apache.spark.sql.DataFrame]]. + * + * NOTE: This is currently private[spark] but will be made public later once it is stabilized. */ +@DeveloperApi private[spark] class VectorUDT extends UserDefinedType[Vector] { override def sqlType: StructType = { @@ -169,6 +175,13 @@ private[spark] class VectorUDT extends UserDefinedType[Vector] { override def pyUDT: String = "pyspark.mllib.linalg.VectorUDT" override def userClass: Class[Vector] = classOf[Vector] + + override def equals(o: Any): Boolean = { + o match { + case v: VectorUDT => true + case _ => false + } + } } /** 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 56a9dbdd58b64..50995ffef9ad5 100644 --- a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java @@ -65,7 +65,7 @@ public void pipeline() { .setStages(new PipelineStage[] {scaler, lr}); PipelineModel model = pipeline.fit(dataset); model.transform(dataset).registerTempTable("prediction"); - DataFrame predictions = jsql.sql("SELECT label, score, prediction FROM prediction"); + DataFrame predictions = jsql.sql("SELECT label, probability, prediction FROM prediction"); predictions.collectAsList(); } } 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 f4ba23c44563e..26284023b0f69 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 @@ -18,17 +18,22 @@ package org.apache.spark.ml.classification; import java.io.Serializable; +import java.lang.Math; import java.util.List; 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 static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.sql.DataFrame; import org.apache.spark.sql.SQLContext; -import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList; +import org.apache.spark.sql.Row; + public class JavaLogisticRegressionSuite implements Serializable { @@ -36,12 +41,17 @@ public class JavaLogisticRegressionSuite implements Serializable { private transient SQLContext jsql; private transient DataFrame dataset; + private transient JavaRDD datasetRDD; + private double eps = 1e-5; + @Before public void setUp() { jsc = new JavaSparkContext("local", "JavaLogisticRegressionSuite"); jsql = new SQLContext(jsc); List points = generateLogisticInputAsList(1.0, 1.0, 100, 42); - dataset = jsql.applySchema(jsc.parallelize(points, 2), LabeledPoint.class); + datasetRDD = jsc.parallelize(points, 2); + dataset = jsql.applySchema(datasetRDD, LabeledPoint.class); + dataset.registerTempTable("dataset"); } @After @@ -51,29 +61,88 @@ public void tearDown() { } @Test - public void logisticRegression() { + public void logisticRegressionDefaultParams() { LogisticRegression lr = new LogisticRegression(); + assert(lr.getLabelCol().equals("label")); LogisticRegressionModel model = lr.fit(dataset); model.transform(dataset).registerTempTable("prediction"); - DataFrame predictions = jsql.sql("SELECT label, score, prediction FROM prediction"); + DataFrame predictions = jsql.sql("SELECT label, probability, prediction FROM prediction"); predictions.collectAsList(); + // Check defaults + assert(model.getThreshold() == 0.5); + assert(model.getFeaturesCol().equals("features")); + assert(model.getPredictionCol().equals("prediction")); + assert(model.getProbabilityCol().equals("probability")); } @Test public void logisticRegressionWithSetters() { + // Set params, train, and check as many params as we can. LogisticRegression lr = new LogisticRegression() .setMaxIter(10) - .setRegParam(1.0); + .setRegParam(1.0) + .setThreshold(0.6) + .setProbabilityCol("myProbability"); LogisticRegressionModel model = lr.fit(dataset); - model.transform(dataset, model.threshold().w(0.8)) // overwrite threshold - .registerTempTable("prediction"); - DataFrame predictions = jsql.sql("SELECT label, score, prediction FROM prediction"); - predictions.collectAsList(); + assert(model.fittingParamMap().apply(lr.maxIter()) == 10); + assert(model.fittingParamMap().apply(lr.regParam()).equals(1.0)); + assert(model.fittingParamMap().apply(lr.threshold()).equals(0.6)); + assert(model.getThreshold() == 0.6); + + // Modify model params, and check that the params worked. + model.setThreshold(1.0); + model.transform(dataset).registerTempTable("predAllZero"); + DataFrame predAllZero = jsql.sql("SELECT prediction, myProbability FROM predAllZero"); + for (Row r: predAllZero.collectAsList()) { + assert(r.getDouble(0) == 0.0); + } + // Call transform with params, and check that the params worked. + model.transform(dataset, model.threshold().w(0.0), model.probabilityCol().w("myProb")) + .registerTempTable("predNotAllZero"); + DataFrame predNotAllZero = jsql.sql("SELECT prediction, myProb FROM predNotAllZero"); + boolean foundNonZero = false; + for (Row r: predNotAllZero.collectAsList()) { + if (r.getDouble(0) != 0.0) foundNonZero = true; + } + assert(foundNonZero); + + // Call fit() with new params, and check as many params as we can. + LogisticRegressionModel model2 = lr.fit(dataset, lr.maxIter().w(5), lr.regParam().w(0.1), + lr.threshold().w(0.4), lr.probabilityCol().w("theProb")); + assert(model2.fittingParamMap().apply(lr.maxIter()) == 5); + assert(model2.fittingParamMap().apply(lr.regParam()).equals(0.1)); + assert(model2.fittingParamMap().apply(lr.threshold()).equals(0.4)); + assert(model2.getThreshold() == 0.4); + assert(model2.getProbabilityCol().equals("theProb")); } + @SuppressWarnings("unchecked") @Test - public void logisticRegressionFitWithVarargs() { + public void logisticRegressionPredictorClassifierMethods() { LogisticRegression lr = new LogisticRegression(); - lr.fit(dataset, lr.maxIter().w(10), lr.regParam().w(1.0)); + LogisticRegressionModel model = lr.fit(dataset); + assert(model.numClasses() == 2); + + model.transform(dataset).registerTempTable("transformed"); + DataFrame trans1 = jsql.sql("SELECT rawPrediction, probability FROM transformed"); + for (Row row: trans1.collect()) { + Vector raw = (Vector)row.get(0); + Vector prob = (Vector)row.get(1); + assert(raw.size() == 2); + assert(prob.size() == 2); + double probFromRaw1 = 1.0 / (1.0 + Math.exp(-raw.apply(1))); + assert(Math.abs(prob.apply(1) - probFromRaw1) < eps); + assert(Math.abs(prob.apply(0) - (1.0 - probFromRaw1)) < eps); + } + + DataFrame trans2 = jsql.sql("SELECT prediction, probability FROM transformed"); + for (Row row: trans2.collect()) { + double pred = row.getDouble(0); + Vector prob = (Vector)row.get(1); + double probOfPred = prob.apply((int)pred); + for (int i = 0; i < prob.size(); ++i) { + assert(probOfPred >= prob.apply(i)); + } + } } } 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 new file mode 100644 index 0000000000000..5bd616e74d86c --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/regression/JavaLinearRegressionSuite.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.Serializable; +import java.util.List; + +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 static org.apache.spark.mllib.classification.LogisticRegressionSuite + .generateLogisticInputAsList; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.SQLContext; + + +public class JavaLinearRegressionSuite implements Serializable { + + private transient JavaSparkContext jsc; + private transient SQLContext jsql; + private transient DataFrame dataset; + private transient JavaRDD datasetRDD; + + @Before + public void setUp() { + jsc = new JavaSparkContext("local", "JavaLinearRegressionSuite"); + 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.registerTempTable("dataset"); + } + + @After + public void tearDown() { + jsc.stop(); + jsc = null; + } + + @Test + public void linearRegressionDefaultParams() { + LinearRegression lr = new LinearRegression(); + assert(lr.getLabelCol().equals("label")); + LinearRegressionModel model = lr.fit(dataset); + model.transform(dataset).registerTempTable("prediction"); + DataFrame predictions = jsql.sql("SELECT label, prediction FROM prediction"); + predictions.collect(); + // Check defaults + assert(model.getFeaturesCol().equals("features")); + assert(model.getPredictionCol().equals("prediction")); + } + + @Test + public void linearRegressionWithSetters() { + // Set params, train, and check as many params as we can. + LinearRegression lr = new LinearRegression() + .setMaxIter(10) + .setRegParam(1.0); + LinearRegressionModel model = lr.fit(dataset); + assert(model.fittingParamMap().apply(lr.maxIter()) == 10); + assert(model.fittingParamMap().apply(lr.regParam()).equals(1.0)); + + // Call fit() with new params, and check as many params as we can. + LinearRegressionModel model2 = + lr.fit(dataset, lr.maxIter().w(5), lr.regParam().w(0.1), lr.predictionCol().w("thePred")); + assert(model2.fittingParamMap().apply(lr.maxIter()) == 5); + assert(model2.fittingParamMap().apply(lr.regParam()).equals(0.1)); + assert(model2.getPredictionCol().equals("thePred")); + } +} 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 33e40dc7410cc..b3d1bfcfbee0f 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 @@ -20,44 +20,108 @@ package org.apache.spark.ml.classification import org.scalatest.FunSuite import org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInput +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.{SQLContext, DataFrame} +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.sql.{DataFrame, Row, SQLContext} + class LogisticRegressionSuite extends FunSuite with MLlibTestSparkContext { @transient var sqlContext: SQLContext = _ @transient var dataset: DataFrame = _ + private val eps: Double = 1e-5 override def beforeAll(): Unit = { super.beforeAll() sqlContext = new SQLContext(sc) dataset = sqlContext.createDataFrame( - sc.parallelize(generateLogisticInput(1.0, 1.0, 100, 42), 2)) + sc.parallelize(generateLogisticInput(1.0, 1.0, nPoints = 100, seed = 42), 2)) } - test("logistic regression") { + test("logistic regression: default params") { val lr = new LogisticRegression + assert(lr.getLabelCol == "label") + assert(lr.getFeaturesCol == "features") + assert(lr.getPredictionCol == "prediction") + assert(lr.getRawPredictionCol == "rawPrediction") + assert(lr.getProbabilityCol == "probability") val model = lr.fit(dataset) model.transform(dataset) - .select("label", "prediction") + .select("label", "probability", "prediction", "rawPrediction") .collect() + assert(model.getThreshold === 0.5) + assert(model.getFeaturesCol == "features") + assert(model.getPredictionCol == "prediction") + assert(model.getRawPredictionCol == "rawPrediction") + assert(model.getProbabilityCol == "probability") } test("logistic regression with setters") { + // Set params, train, and check as many params as we can. val lr = new LogisticRegression() .setMaxIter(10) .setRegParam(1.0) + .setThreshold(0.6) + .setProbabilityCol("myProbability") val model = lr.fit(dataset) - model.transform(dataset, model.threshold -> 0.8) // overwrite threshold - .select("label", "score", "prediction") + assert(model.fittingParamMap.get(lr.maxIter) === Some(10)) + assert(model.fittingParamMap.get(lr.regParam) === Some(1.0)) + assert(model.fittingParamMap.get(lr.threshold) === Some(0.6)) + assert(model.getThreshold === 0.6) + + // Modify model params, and check that the params worked. + model.setThreshold(1.0) + val predAllZero = model.transform(dataset) + .select("prediction", "myProbability") .collect() + .map { case Row(pred: Double, prob: Vector) => pred } + assert(predAllZero.forall(_ === 0), + s"With threshold=1.0, expected predictions to be all 0, but only" + + s" ${predAllZero.count(_ === 0)} of ${dataset.count()} were 0.") + // Call transform with params, and check that the params worked. + val predNotAllZero = + model.transform(dataset, model.threshold -> 0.0, model.probabilityCol -> "myProb") + .select("prediction", "myProb") + .collect() + .map { case Row(pred: Double, prob: Vector) => pred } + assert(predNotAllZero.exists(_ !== 0.0)) + + // Call fit() with new params, and check as many params as we can. + val model2 = lr.fit(dataset, lr.maxIter -> 5, lr.regParam -> 0.1, lr.threshold -> 0.4, + lr.probabilityCol -> "theProb") + assert(model2.fittingParamMap.get(lr.maxIter).get === 5) + assert(model2.fittingParamMap.get(lr.regParam).get === 0.1) + assert(model2.fittingParamMap.get(lr.threshold).get === 0.4) + assert(model2.getThreshold === 0.4) + assert(model2.getProbabilityCol == "theProb") } - test("logistic regression fit and transform with varargs") { + test("logistic regression: Predictor, Classifier methods") { + val sqlContext = this.sqlContext val lr = new LogisticRegression - val model = lr.fit(dataset, lr.maxIter -> 10, lr.regParam -> 1.0) - model.transform(dataset, model.threshold -> 0.8, model.scoreCol -> "probability") - .select("label", "probability", "prediction") - .collect() + + val model = lr.fit(dataset) + assert(model.numClasses === 2) + + val threshold = model.getThreshold + val results = model.transform(dataset) + + // Compare rawPrediction with probability + results.select("rawPrediction", "probability").collect().map { + case Row(raw: Vector, prob: Vector) => + assert(raw.size === 2) + assert(prob.size === 2) + val probFromRaw1 = 1.0 / (1.0 + math.exp(-raw(1))) + assert(prob(1) ~== probFromRaw1 relTol eps) + assert(prob(0) ~== 1.0 - probFromRaw1 relTol eps) + } + + // Compare prediction with probability + results.select("prediction", "probability").collect().map { + case Row(pred: Double, prob: Vector) => + val predFromProb = prob.toArray.zipWithIndex.maxBy(_._1)._2 + assert(pred == predFromProb) + } } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala new file mode 100644 index 0000000000000..bbb44c3e2dfc2 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.mllib.classification.LogisticRegressionSuite.generateLogisticInput +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.{DataFrame, SQLContext} + +class LinearRegressionSuite extends FunSuite with MLlibTestSparkContext { + + @transient var sqlContext: SQLContext = _ + @transient var dataset: DataFrame = _ + + override def beforeAll(): Unit = { + super.beforeAll() + sqlContext = new SQLContext(sc) + dataset = sqlContext.createDataFrame( + sc.parallelize(generateLogisticInput(1.0, 1.0, nPoints = 100, seed = 42), 2)) + } + + test("linear regression: default params") { + val lr = new LinearRegression + assert(lr.getLabelCol == "label") + val model = lr.fit(dataset) + model.transform(dataset) + .select("label", "prediction") + .collect() + // Check defaults + assert(model.getFeaturesCol == "features") + assert(model.getPredictionCol == "prediction") + } + + test("linear regression with setters") { + // Set params, train, and check as many as we can. + val lr = new LinearRegression() + .setMaxIter(10) + .setRegParam(1.0) + val model = lr.fit(dataset) + assert(model.fittingParamMap.get(lr.maxIter).get === 10) + assert(model.fittingParamMap.get(lr.regParam).get === 1.0) + + // Call fit() with new params, and check as many as we can. + val model2 = lr.fit(dataset, lr.maxIter -> 5, lr.regParam -> 0.1, lr.predictionCol -> "thePred") + assert(model2.fittingParamMap.get(lr.maxIter).get === 5) + assert(model2.fittingParamMap.get(lr.regParam).get === 0.1) + assert(model2.getPredictionCol == "thePred") + } +} diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index b17532c1d814c..4065a562a1a18 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -36,6 +36,7 @@ object MimaExcludes { case v if v.startsWith("1.3") => Seq( MimaBuild.excludeSparkPackage("deploy"), + MimaBuild.excludeSparkPackage("ml"), // These are needed if checking against the sbt build, since they are part of // the maven-generated artifacts in the 1.2 build. MimaBuild.excludeSparkPackage("unused"), @@ -142,6 +143,11 @@ object MimaExcludes { "org.apache.spark.graphx.Graph.getCheckpointFiles"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.graphx.Graph.isCheckpointed") + ) ++ Seq( + // SPARK-4789 Standardize ML Prediction APIs + 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") ) case v if v.startsWith("1.2") => From 575d2df350e32964fffb02536cb0587a92ef20e9 Mon Sep 17 00:00:00 2001 From: Leolh Date: Fri, 6 Feb 2015 09:01:53 +0000 Subject: [PATCH 108/578] [SPARK-5380][GraphX] Solve an ArrayIndexOutOfBoundsException when build graph with a file format error When I build a graph with a file format error, there will be an ArrayIndexOutOfBoundsException Author: Leolh Closes #4176 from Leolh/patch-1 and squashes the following commits: 94f6d22 [Leolh] Update GraphLoader.scala 23767f1 [Leolh] [SPARK-3650][GraphX] There will be an ArrayIndexOutOfBoundsException if the format of the source file is wrong --- graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala index 4933aecba1286..21187be7678a6 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala @@ -77,7 +77,7 @@ object GraphLoader extends Logging { if (!line.isEmpty && line(0) != '#') { val lineArray = line.split("\\s+") if (lineArray.length < 2) { - logWarning("Invalid line: " + line) + throw new IllegalArgumentException("Invalid line: " + line) } val srcId = lineArray(0).toLong val dstId = lineArray(1).toLong From f6ba813af21f8aa277155b789a7330e356ce15da Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 6 Feb 2015 09:33:36 +0000 Subject: [PATCH 109/578] [Minor] Remove permission for execution from spark-shell.cmd .cmd files in bin is not set permission for execution except for spark-shell.cmd. Let's unify that. Author: Kousuke Saruta Closes #3983 from sarutak/fix-mode-of-cmd and squashes the following commits: 9d6eedc [Kousuke Saruta] Removed permission for execution from spark-shell.cmd --- bin/spark-shell.cmd | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100755 => 100644 bin/spark-shell.cmd diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd old mode 100755 new mode 100644 From 24dbc50b934bfcd9fc3fd91cfd9966b75b250ad7 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 6 Feb 2015 09:39:12 +0000 Subject: [PATCH 110/578] [SPARK-5157][YARN] Configure more JVM options properly when we use ConcMarkSweepGC for AM. When we set `SPARK_USE_CONC_INCR_GC`, ConcurrentMarkSweepGC works on the AM. Actually, if ConcurrentMarkSweepGC is set for the JVM, following JVM options are set automatically and implicitly. * MaxTenuringThreshold=0 * SurvivorRatio=1024 Those can not be proper value for most cases. See also http://www.oracle.com/technetwork/java/tuning-139912.html Author: Kousuke Saruta Closes #3956 from sarutak/SPARK-5157 and squashes the following commits: c15da4e [Kousuke Saruta] Set more JVM options for AM when enabling CMS --- yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 2 ++ 1 file changed, 2 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 91e8574e94e2f..e7005094b5f3c 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 @@ -418,6 +418,8 @@ private[spark] class Client( // In our expts, using (default) throughput collector has severe perf ramifications in // multi-tenant machines javaOpts += "-XX:+UseConcMarkSweepGC" + javaOpts += "-XX:MaxTenuringThreshold=31" + javaOpts += "-XX:SurvivorRatio=8" javaOpts += "-XX:+CMSIncrementalMode" javaOpts += "-XX:+CMSIncrementalPacing" javaOpts += "-XX:CMSIncrementalDutyCycleMin=0" From 856928979f7d00fbb518fc1102a2e7d80cea0f7c Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 6 Feb 2015 10:07:20 +0000 Subject: [PATCH 111/578] [SPARK-5582] [history] Ignore empty log directories. Empty log directories are not useful at the moment, but if one ends up showing in the log root, it breaks the code that checks for log directories. Author: Marcelo Vanzin Closes #4352 from vanzin/SPARK-5582 and squashes the following commits: 1a6a3d4 [Marcelo Vanzin] [SPARK-5582] Fix exception when looking at empty directories. --- .../deploy/history/FsHistoryProvider.scala | 22 ++++++++++++------- .../history/FsHistoryProviderSuite.scala | 18 +++++++++++++++ 2 files changed, 32 insertions(+), 8 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 0ae45f4ad9130..92125f2df7d10 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 @@ -173,9 +173,10 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis val logInfos = statusList .filter { entry => try { - val modTime = getModificationTime(entry) - newLastModifiedTime = math.max(newLastModifiedTime, modTime) - modTime >= lastModifiedTime + getModificationTime(entry).map { time => + newLastModifiedTime = math.max(newLastModifiedTime, time) + time >= lastModifiedTime + }.getOrElse(false) } catch { case e: AccessControlException => // Do not use "logInfo" since these messages can get pretty noisy if printed on @@ -251,7 +252,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis appListener.appName.getOrElse(NOT_STARTED), appListener.startTime.getOrElse(-1L), appListener.endTime.getOrElse(-1L), - getModificationTime(eventLog), + getModificationTime(eventLog).get, appListener.sparkUser.getOrElse(NOT_STARTED), isApplicationCompleted(eventLog)) } finally { @@ -310,11 +311,16 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis */ private def isLegacyLogDirectory(entry: FileStatus): Boolean = entry.isDir() - private def getModificationTime(fsEntry: FileStatus): Long = { - if (fsEntry.isDir) { - fs.listStatus(fsEntry.getPath).map(_.getModificationTime()).max + /** + * Returns the modification time of the given event log. If the status points at an empty + * directory, `None` is returned, indicating that there isn't an event log at that location. + */ + private def getModificationTime(fsEntry: FileStatus): Option[Long] = { + if (isLegacyLogDirectory(fsEntry)) { + val statusList = fs.listStatus(fsEntry.getPath) + if (!statusList.isEmpty) Some(statusList.map(_.getModificationTime()).max) else None } else { - fsEntry.getModificationTime() + Some(fsEntry.getModificationTime()) } } 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 3fbc1a21d10ed..1d95432258111 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 @@ -190,6 +190,24 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers appListAfterRename.head.logPath should not endWith(EventLoggingListener.IN_PROGRESS) } + test("SPARK-5582: empty log directory") { + val conf = new SparkConf() + .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) + val provider = new FsHistoryProvider(conf) + + val logFile1 = new File(testDir, "app1" + EventLoggingListener.IN_PROGRESS) + writeFile(logFile1, true, None, + SparkListenerApplicationStart("app1", Some("app1"), 1L, "test"), + SparkListenerApplicationEnd(2L)) + + val oldLog = new File(testDir, "old1") + oldLog.mkdir() + + provider.checkForLogs() + val appListAfterRename = provider.getListing() + appListAfterRename.size should be (1) + } + private def writeFile(file: File, isNewFormat: Boolean, codec: Option[CompressionCodec], events: SparkListenerEvent*) = { val out = From ed3aac791232560d2cfe25e3304e48873d48b338 Mon Sep 17 00:00:00 2001 From: lianhuiwang Date: Fri, 6 Feb 2015 11:00:35 +0000 Subject: [PATCH 112/578] [SPARK-5470][Core]use defaultClassLoader to load classes in KryoSerializer Now KryoSerializer load classes of classesToRegister at the time of its initialization. when we set spark.kryo.classesToRegister=class1, it will throw SparkException("Failed to load class to register with Kryo". because in KryoSerializer's initialization, classLoader cannot include class of user's jars. we need to use defaultClassLoader of Serializer in newKryo(), because executor will reset defaultClassLoader of Serializer after Serializer's initialization. thank zzcclp for reporting it to me. Author: lianhuiwang Closes #4258 from lianhuiwang/SPARK-5470 and squashes the following commits: 73b719f [lianhuiwang] do the splitting and filtering during initialization 64cf306 [lianhuiwang] use defaultClassLoader to load classes of classesToRegister in KryoSerializer --- .../org/apache/spark/serializer/KryoSerializer.scala | 11 ++--------- 1 file changed, 2 insertions(+), 9 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 d56e23ce4478a..02158aa0f866e 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -58,14 +58,6 @@ class KryoSerializer(conf: SparkConf) private val classesToRegister = conf.get("spark.kryo.classesToRegister", "") .split(',') .filter(!_.isEmpty) - .map { className => - try { - Class.forName(className) - } catch { - case e: Exception => - throw new SparkException("Failed to load class to register with Kryo", e) - } - } def newKryoOutput() = new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize)) @@ -97,7 +89,8 @@ class KryoSerializer(conf: SparkConf) // Use the default classloader when calling the user registrator. Thread.currentThread.setContextClassLoader(classLoader) // Register classes given through spark.kryo.classesToRegister. - classesToRegister.foreach { clazz => kryo.register(clazz) } + classesToRegister + .foreach { className => kryo.register(Class.forName(className, true, classLoader)) } // Allow the user to register their own classes by setting spark.kryo.registrator. userRegistrator .map(Class.forName(_, true, classLoader).newInstance().asInstanceOf[KryoRegistrator]) From f827ef4d7ead85aab65837ea625141e05e917c2e Mon Sep 17 00:00:00 2001 From: Miguel Peralvo Date: Fri, 6 Feb 2015 11:04:48 +0000 Subject: [PATCH 113/578] Update ec2-scripts.md Change spark-version from 1.1.0 to 1.2.0 in the example for spark-ec2/Launch Cluster. Author: Miguel Peralvo Closes #4300 from MiguelPeralvo/patch-1 and squashes the following commits: 38adf0b [Miguel Peralvo] Update ec2-scripts.md 1850869 [Miguel Peralvo] Update ec2-scripts.md --- docs/ec2-scripts.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md index d50f445d7ecc7..8c9a1e1262d8f 100644 --- a/docs/ec2-scripts.md +++ b/docs/ec2-scripts.md @@ -52,7 +52,7 @@ identify machines belonging to each cluster in the Amazon EC2 Console. ```bash export AWS_SECRET_ACCESS_KEY=AaBbCcDdEeFGgHhIiJjKkLlMmNnOoPpQqRrSsTtU export AWS_ACCESS_KEY_ID=ABCDEFG1234567890123 -./spark-ec2 --key-pair=awskey --identity-file=awskey.pem --region=us-west-1 --zone=us-west-1a --spark-version=1.1.0 launch my-spark-cluster +./spark-ec2 --key-pair=awskey --identity-file=awskey.pem --region=us-west-1 --zone=us-west-1a launch my-spark-cluster ``` - After everything launches, check that the cluster scheduler is up and sees From cf6778e8d8e8d7e0b4e7b17637ad812624a937dd Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Fri, 6 Feb 2015 11:38:39 +0000 Subject: [PATCH 114/578] [Build] Set all Debian package permissions to 755 755 means the owner can read, write, and execute, and everyone else can just read and execute. I think that's what we want here since without execute permissions others cannot open directories. Inspired by [this comment on a separate PR](https://github.com/apache/spark/pull/3297#issuecomment-63286730). Author: Nicholas Chammas Closes #4277 from nchammas/patch-1 and squashes the following commits: da77fb0 [Nicholas Chammas] [Build] Set all Debian package permissions to 755 --- assembly/pom.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 1bb5a671f5390..301ff69c2ae3b 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -39,7 +39,7 @@ spark /usr/share/spark root - 744 + 755 @@ -280,7 +280,7 @@ ${deb.user} ${deb.user} ${deb.install.path}/conf - 744 + ${deb.bin.filemode} @@ -302,7 +302,7 @@ ${deb.user} ${deb.user} ${deb.install.path}/sbin - 744 + ${deb.bin.filemode} @@ -313,7 +313,7 @@ ${deb.user} ${deb.user} ${deb.install.path}/python - 744 + ${deb.bin.filemode} From 37d35ab53b82916e4bf0eeb788044bfc33f329df Mon Sep 17 00:00:00 2001 From: Ryan Williams Date: Fri, 6 Feb 2015 12:22:25 +0000 Subject: [PATCH 115/578] [SPARK-5416] init Executor.threadPool before ExecutorSource Some ExecutorSource metrics can NPE by attempting to reference the threadpool otherwise. Author: Ryan Williams Closes #4212 from ryan-williams/threadpool and squashes the following commits: 236f2ad [Ryan Williams] init Executor.threadPool before ExecutorSource --- .../src/main/scala/org/apache/spark/executor/Executor.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 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 312bb3a1daaa3..5141483d1e745 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -75,6 +75,9 @@ private[spark] class Executor( Thread.setDefaultUncaughtExceptionHandler(SparkUncaughtExceptionHandler) } + // Start worker thread pool + val threadPool = Utils.newDaemonCachedThreadPool("Executor task launch worker") + val executorSource = new ExecutorSource(this, executorId) if (!isLocal) { @@ -101,9 +104,6 @@ private[spark] class Executor( // Limit of bytes for total size of results (default is 1GB) private val maxResultSize = Utils.getMaxResultSize(conf) - // Start worker thread pool - val threadPool = Utils.newDaemonCachedThreadPool("Executor task launch worker") - // Maintains the list of running tasks. private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] From 9ad56ad2a2a51df449040c4f4b7c66b104883312 Mon Sep 17 00:00:00 2001 From: Travis Galoppo Date: Fri, 6 Feb 2015 10:26:51 -0800 Subject: [PATCH 116/578] [SPARK-5013] [MLlib] Added documentation and sample data file for GaussianMixture Simple description and code samples (and sample data) for GaussianMixture Author: Travis Galoppo Closes #4401 from tgaloppo/spark-5013 and squashes the following commits: c9ff9a5 [Travis Galoppo] Fixed link in mllib-clustering.md Added Gaussian mixture and power iteration as available clustering techniques in mllib-guide 2368690 [Travis Galoppo] Minor fixes 3eb41fa [Travis Galoppo] [SPARK-5013] Added documentation and sample data file for GaussianMixture --- data/mllib/gmm_data.txt | 2000 ++++++++++++++++++++++++++++++++++++++ docs/mllib-clustering.md | 131 ++- docs/mllib-guide.md | 2 + 3 files changed, 2130 insertions(+), 3 deletions(-) create mode 100644 data/mllib/gmm_data.txt diff --git a/data/mllib/gmm_data.txt b/data/mllib/gmm_data.txt new file mode 100644 index 0000000000000..934ee4a83a2df --- /dev/null +++ b/data/mllib/gmm_data.txt @@ -0,0 +1,2000 @@ + 2.59470454e+00 2.12298217e+00 + 1.15807024e+00 -1.46498723e-01 + 2.46206638e+00 6.19556894e-01 + -5.54845070e-01 -7.24700066e-01 + -3.23111426e+00 -1.42579084e+00 + 3.02978115e+00 7.87121753e-01 + 1.97365907e+00 1.15914704e+00 + -6.44852101e+00 -3.18154314e+00 + 1.30963349e+00 1.62866434e-01 + 4.26482541e+00 2.15547996e+00 + 3.79927257e+00 1.50572445e+00 + 4.17452609e-01 -6.74032760e-01 + 4.21117627e-01 4.45590255e-01 + -2.80425571e+00 -7.77150554e-01 + 2.55928797e+00 7.03954218e-01 + 1.32554059e+00 -9.46663152e-01 + -3.39691439e+00 -1.49005743e+00 + -2.26542270e-01 3.60052515e-02 + 1.04994198e+00 5.29825685e-01 + -1.51566882e+00 -1.86264432e-01 + -3.27928172e-01 -7.60859110e-01 + -3.18054866e-01 3.97719805e-01 + 1.65579418e-01 -3.47232033e-01 + 6.47162333e-01 4.96059961e-02 + -2.80776647e-01 4.79418757e-01 + 7.45069752e-01 1.20790281e-01 + 2.13604102e-01 1.59542555e-01 + -3.08860224e+00 -1.43259870e+00 + 8.97066497e-01 1.10206801e+00 + -2.23918874e-01 -1.07267267e+00 + 2.51525708e+00 2.84761973e-01 + 9.98052532e-01 1.08333783e+00 + 1.76705588e+00 8.18866778e-01 + 5.31555163e-02 -1.90111151e-01 + -2.17405059e+00 7.21854582e-02 + -2.13772505e+00 -3.62010387e-01 + 2.95974057e+00 1.31602381e+00 + 2.74053561e+00 1.61781757e+00 + 6.68135448e-01 2.86586009e-01 + 2.82323739e+00 1.74437257e+00 + 8.11540288e-01 5.50744478e-01 + 4.10050897e-01 5.10668402e-03 + 9.58626136e-01 -3.49633680e-01 + 4.66599798e+00 1.49964894e+00 + 4.94507794e-01 2.58928077e-01 + -2.36029742e+00 -1.61042909e+00 + -4.99306804e-01 -8.04984769e-01 + 1.07448510e+00 9.39605828e-01 + -1.80448949e+00 -1.05983264e+00 + -3.22353821e-01 1.73612093e-01 + 1.85418702e+00 1.15640643e+00 + 6.93794163e-01 6.59993560e-01 + 1.99399102e+00 1.44547123e+00 + 3.38866124e+00 1.23379290e+00 + -4.24067720e+00 -1.22264282e+00 + 6.03230201e-02 2.95232729e-01 + -3.59341813e+00 -7.17453726e-01 + 4.87447372e-01 -2.00733911e-01 + 1.20149195e+00 4.07880197e-01 + -2.13331464e+00 -4.58518077e-01 + -3.84091083e+00 -1.71553950e+00 + -5.37279250e-01 2.64822629e-02 + -2.10155227e+00 -1.32558103e+00 + -1.71318897e+00 -7.12098563e-01 + -1.46280695e+00 -1.84868337e-01 + -3.59785325e+00 -1.54832434e+00 + -5.77528081e-01 -5.78580857e-01 + 3.14734283e-01 5.80184639e-01 + -2.71164714e+00 -1.19379432e+00 + 1.09634489e+00 7.20143887e-01 + -3.05527722e+00 -1.47774064e+00 + 6.71753586e-01 7.61350020e-01 + 3.98294144e+00 1.54166484e+00 + -3.37220384e+00 -2.21332064e+00 + 1.81222914e+00 7.41212752e-01 + 2.71458282e-01 1.36329078e-01 + -3.97815359e-01 1.16766886e-01 + -1.70192814e+00 -9.75851571e-01 + -3.46803804e+00 -1.09965988e+00 + -1.69649627e+00 -5.76045801e-01 + -1.02485636e-01 -8.81841246e-01 + -3.24194667e-02 2.55429276e-01 + -2.75343168e+00 -1.51366320e+00 + -2.78676702e+00 -5.22360489e-01 + 1.70483164e+00 1.19769805e+00 + 4.92022579e-01 3.24944706e-01 + 2.48768464e+00 1.00055363e+00 + 4.48786400e-01 7.63902870e-01 + 2.93862696e+00 1.73809968e+00 + -3.55019305e+00 -1.97875558e+00 + 1.74270784e+00 6.90229224e-01 + 5.13391994e-01 4.58374016e-01 + 1.78379499e+00 9.08026381e-01 + 1.75814147e+00 7.41449784e-01 + -2.30687792e-01 3.91009729e-01 + 3.92271353e+00 1.44006290e+00 + 2.93361679e-01 -4.99886375e-03 + 2.47902690e-01 -7.49542503e-01 + -3.97675355e-01 1.36824887e-01 + 3.56535953e+00 1.15181329e+00 + 3.22425301e+00 1.28702383e+00 + -2.94192478e-01 -2.42382557e-01 + 8.02068864e-01 -1.51671475e-01 + 8.54133530e-01 -4.89514885e-02 + -1.64316316e-01 -5.34642346e-01 + -6.08485405e-01 -2.10332352e-01 + -2.18940059e+00 -1.07024952e+00 + -1.71586960e+00 -2.83333492e-02 + 1.70200448e-01 -3.28031178e-01 + -1.97210346e+00 -5.39948532e-01 + 2.19500160e+00 1.05697170e+00 + -1.76239935e+00 -1.09377438e+00 + 1.68314744e+00 6.86491164e-01 + -2.99852288e+00 -1.46619067e+00 + -2.23769560e+00 -9.15008355e-01 + 9.46887516e-01 5.58410503e-01 + 5.02153123e-01 1.63851235e-01 + -9.70297062e-01 3.14625374e-01 + -1.29405593e+00 -8.20994131e-01 + 2.72516079e+00 7.85839947e-01 + 1.45788024e+00 3.37487353e-01 + -4.36292749e-01 -5.42150480e-01 + 2.21304711e+00 1.25254042e+00 + -1.20810271e-01 4.79632898e-01 + -3.30884511e+00 -1.50607586e+00 + -6.55882455e+00 -1.94231256e+00 + -3.17033630e+00 -9.94678930e-01 + 1.42043617e+00 7.28808957e-01 + -1.57546099e+00 -1.10320497e+00 + -3.22748754e+00 -1.64174579e+00 + 2.96776017e-03 -3.16191512e-02 + -2.25986054e+00 -6.13123197e-01 + 2.49434243e+00 7.73069183e-01 + 9.08494049e-01 -1.53926853e-01 + -2.80559090e+00 -1.37474221e+00 + 4.75224286e-01 2.53153674e-01 + 4.37644006e+00 8.49116998e-01 + 2.27282959e+00 6.16568202e-01 + 1.16006880e+00 1.65832798e-01 + -1.67163193e+00 -1.22555386e+00 + -1.38231118e+00 -7.29575504e-01 + -3.49922750e+00 -2.26446675e+00 + -3.73780110e-01 -1.90657869e-01 + 1.68627679e+00 1.05662987e+00 + -3.28891792e+00 -1.11080334e+00 + -2.59815798e+00 -1.51410198e+00 + -2.61203309e+00 -6.00143552e-01 + 6.58964943e-01 4.47216094e-01 + -2.26711381e+00 -7.26512923e-01 + -5.31429009e-02 -1.97925341e-02 + 3.19749807e+00 9.20425476e-01 + -1.37595787e+00 -6.58062732e-01 + 8.09900278e-01 -3.84286160e-01 + -5.07741280e+00 -1.97683808e+00 + -2.99764250e+00 -1.50753777e+00 + -9.87671815e-01 -4.63255889e-01 + 1.65390765e+00 6.73806615e-02 + 5.51252659e+00 2.69842267e+00 + -2.23724309e+00 -4.77624004e-01 + 4.99726228e+00 1.74690949e+00 + 1.75859162e-01 -1.49350995e-01 + 4.13382789e+00 1.31735161e+00 + 2.69058117e+00 4.87656923e-01 + 1.07180318e+00 1.01426954e+00 + 3.37216869e+00 1.05955377e+00 + -2.95006781e+00 -1.57048303e+00 + -2.46401648e+00 -8.37056374e-01 + 1.19012962e-01 7.54702770e-01 + 3.34142539e+00 4.81938295e-01 + 2.92643913e+00 1.04301050e+00 + 2.89697751e+00 1.37551442e+00 + -1.03094242e+00 2.20903962e-01 + -5.13914589e+00 -2.23355387e+00 + -8.81680780e-01 1.83590000e-01 + 2.82334775e+00 1.26650464e+00 + -2.81042540e-01 -3.26370240e-01 + 2.97995487e+00 8.34569452e-01 + -1.39857135e+00 -1.15798385e+00 + 4.27186506e+00 9.04253702e-01 + 6.98684517e-01 7.91167305e-01 + 3.52233095e+00 1.29976473e+00 + 2.21448029e+00 2.73213379e-01 + -3.13505683e-01 -1.20593774e-01 + 3.70571571e+00 1.06220876e+00 + 9.83881041e-01 5.67713803e-01 + -2.17897705e+00 2.52925205e-01 + 1.38734039e+00 4.61287066e-01 + -1.41181602e+00 -1.67248955e-02 + -1.69974639e+00 -7.17812071e-01 + -2.01005793e-01 -7.49662056e-01 + 1.69016336e+00 3.24687979e-01 + -2.03250179e+00 -2.76108460e-01 + 3.68776848e-01 4.12536941e-01 + 7.66238259e-01 -1.84750637e-01 + -2.73989147e-01 -1.72817250e-01 + -2.18623745e+00 -2.10906798e-01 + -1.39795625e-01 3.26066094e-02 + -2.73826912e-01 -6.67586097e-02 + -1.57880654e+00 -4.99395900e-01 + 4.55950908e+00 2.29410489e+00 + -7.36479631e-01 -1.57861857e-01 + 1.92082888e+00 1.05843391e+00 + 4.29192810e+00 1.38127810e+00 + 1.61852879e+00 1.95871986e-01 + -1.95027403e+00 -5.22448168e-01 + -1.67446281e+00 -9.41497162e-01 + 6.07097859e-01 3.44178029e-01 + -3.44004683e+00 -1.49258461e+00 + 2.72114752e+00 6.00728991e-01 + 8.80685522e-01 -2.53243336e-01 + 1.39254928e+00 3.42988512e-01 + 1.14194836e-01 -8.57945694e-02 + -1.49387332e+00 -7.60860481e-01 + -1.98053285e+00 -4.86039865e-01 + 3.56008568e+00 1.08438692e+00 + 2.27833961e-01 1.09441881e+00 + -1.16716710e+00 -6.54778242e-01 + 2.02156613e+00 5.42075758e-01 + 1.08429178e+00 -7.67420693e-01 + 6.63058455e-01 4.61680991e-01 + -1.06201537e+00 1.38862846e-01 + 3.08701875e+00 8.32580273e-01 + -4.96558108e-01 -2.47031257e-01 + 7.95109987e-01 7.59314147e-02 + -3.39903524e-01 8.71565566e-03 + 8.68351357e-01 4.78358641e-01 + 1.48750819e+00 7.63257420e-01 + -4.51224101e-01 -4.44056898e-01 + -3.02734750e-01 -2.98487961e-01 + 5.46846609e-01 7.02377629e-01 + 1.65129778e+00 3.74008231e-01 + -7.43336512e-01 3.95723531e-01 + -5.88446605e-01 -6.47520211e-01 + 3.58613167e+00 1.95024937e+00 + 3.11718883e+00 8.37984715e-01 + 1.80919244e+00 9.62644986e-01 + 5.43856371e-02 -5.86297543e-01 + -1.95186766e+00 -1.02624212e-01 + 8.95628057e-01 5.91812281e-01 + 4.97691627e-02 5.31137156e-01 + -1.07633113e+00 -2.47392788e-01 + -1.17257986e+00 -8.68528265e-01 + -8.19227665e-02 5.80579434e-03 + -2.86409787e-01 1.95812924e-01 + 1.10582671e+00 7.42853240e-01 + 4.06429774e+00 1.06557476e+00 + -3.42521792e+00 -7.74327139e-01 + 1.28468671e+00 6.20431661e-01 + 6.01201008e-01 -1.16799728e-01 + -1.85058727e-01 -3.76235293e-01 + 5.44083324e+00 2.98490868e+00 + 2.69273070e+00 7.83901153e-01 + 1.88938036e-01 -4.83222152e-01 + 1.05667256e+00 -2.57003165e-01 + 2.99711662e-01 -4.33131912e-01 + 7.73689216e-02 -1.78738364e-01 + 9.58326279e-01 6.38325706e-01 + -3.97727049e-01 2.27314759e-01 + 3.36098175e+00 1.12165237e+00 + 1.77804871e+00 6.46961933e-01 + -2.86945546e+00 -1.00395518e+00 + 3.03494815e+00 7.51814612e-01 + -1.43658194e+00 -3.55432244e-01 + -3.08455105e+00 -1.51535106e+00 + -1.55841975e+00 3.93454820e-02 + 7.96073412e-01 -3.11036969e-01 + -9.84125401e-01 -1.02064649e+00 + -7.75688143e+00 -3.65219926e+00 + 1.53816429e+00 7.65926670e-01 + -4.92712738e-01 2.32244240e-02 + -1.93166919e+00 -1.07701304e+00 + 2.03029875e-02 -7.54055699e-01 + 2.52177489e+00 1.01544979e+00 + 3.65109048e-01 -9.48328494e-01 + -1.28849143e-01 2.51947174e-01 + -1.02428075e+00 -9.37767116e-01 + -3.04179748e+00 -9.97926994e-01 + -2.51986980e+00 -1.69117413e+00 + -1.24900838e+00 -4.16179917e-01 + 2.77943992e+00 1.22842327e+00 + -4.37434557e+00 -1.70182693e+00 + -1.60019319e+00 -4.18345639e-01 + -1.67613646e+00 -9.44087262e-01 + -9.00843245e-01 8.26378089e-02 + 3.29770621e-01 -9.07870444e-01 + -2.84650535e+00 -9.00155396e-01 + 1.57111705e+00 7.07432268e-01 + 1.24948552e+00 1.04812849e-01 + 1.81440558e+00 9.53545082e-01 + -1.74915794e+00 -1.04606288e+00 + 1.20593269e+00 -1.12607147e-02 + 1.36004919e-01 -1.09828044e+00 + 2.57480693e-01 3.34941541e-01 + 7.78775385e-01 -5.32494732e-01 + -1.79155126e+00 -6.29994129e-01 + -1.75706839e+00 -8.35100126e-01 + 4.29512012e-01 7.81426910e-02 + 3.08349370e-01 -1.27359861e-01 + 1.05560329e+00 4.55150640e-01 + 1.95662574e+00 1.17593217e+00 + 8.77376632e-01 6.57866662e-01 + 7.71311255e-01 9.15134334e-02 + -6.36978275e+00 -2.55874241e+00 + -2.98335339e+00 -1.59567024e+00 + -3.67104587e-01 1.85315291e-01 + 1.95347407e+00 -7.15503113e-02 + 8.45556363e-01 6.51256415e-02 + 9.42868521e-01 3.56647624e-01 + 2.99321875e+00 1.07505254e+00 + -2.91030538e-01 -3.77637183e-01 + 1.62870918e+00 3.37563671e-01 + 2.05773173e-01 3.43337416e-01 + -8.40879199e-01 -1.35600767e-01 + 1.38101624e+00 5.99253495e-01 + -6.93715607e+00 -2.63580662e+00 + -1.04423404e+00 -8.32865050e-01 + 1.33448476e+00 1.04863475e+00 + 6.01675207e-01 1.98585194e-01 + 2.31233993e+00 7.98628331e-01 + 1.85201313e-01 -1.76070247e+00 + 1.92006354e+00 8.45737582e-01 + 1.06320415e+00 2.93426068e-01 + -1.20360141e+00 -1.00301288e+00 + 1.95926629e+00 6.26643532e-01 + 6.04483978e-02 5.72643059e-01 + -1.04568563e+00 -5.91021496e-01 + 2.62300678e+00 9.50997831e-01 + -4.04610275e-01 3.73150879e-01 + 2.26371902e+00 8.73627529e-01 + 2.12545313e+00 7.90640352e-01 + 7.72181917e-03 1.65718952e-02 + 1.00422340e-01 -2.05562936e-01 + -1.22989802e+00 -1.01841681e-01 + 3.09064082e+00 1.04288010e+00 + 5.18274167e+00 1.34749259e+00 + -8.32075153e-01 -1.97592029e-01 + 3.84126764e-02 5.58171345e-01 + 4.99560727e-01 -4.26154438e-02 + 4.79071151e+00 2.19728942e+00 + -2.78437968e+00 -1.17812590e+00 + -2.22804226e+00 -4.31174255e-01 + 8.50762292e-01 -1.06445261e-01 + 1.10812830e+00 -2.59118812e-01 + -2.91450155e-01 6.42802679e-01 + -1.38631532e-01 -5.88585623e-01 + -5.04120983e-01 -2.17094915e-01 + 3.41410820e+00 1.67897767e+00 + -2.23697326e+00 -6.62735244e-01 + -3.55961064e-01 -1.27647226e-01 + -3.55568274e+00 -2.49011369e+00 + -8.77586408e-01 -9.38268065e-03 + 1.52382384e-01 -5.62155760e-01 + 1.55885574e-01 1.07617069e-01 + -8.37129973e-01 -5.22259081e-01 + -2.92741750e+00 -1.35049428e+00 + -3.54670781e-01 5.69205952e-02 + 2.21030255e+00 1.34689986e+00 + 1.60787722e+00 5.75984706e-01 + 1.32294221e+00 5.31577509e-01 + 7.05672928e-01 3.34241244e-01 + 1.41406179e+00 1.15783408e+00 + -6.92172228e-01 -2.84817896e-01 + 3.28358655e-01 -2.66910083e-01 + 1.68013644e-01 -4.28016549e-02 + 2.07365974e+00 7.76496211e-01 + -3.92974907e-01 2.46796730e-01 + -5.76078636e-01 3.25676963e-01 + -1.82547204e-01 -5.06410543e-01 + 3.04754906e+00 1.16174496e+00 + -3.01090632e+00 -1.09195183e+00 + -1.44659696e+00 -6.87838682e-01 + 2.11395861e+00 9.10495785e-01 + 1.40962871e+00 1.13568678e+00 + -1.66653234e-01 -2.10012503e-01 + 3.17456029e+00 9.74502922e-01 + 2.15944820e+00 8.62807189e-01 + -3.45418719e+00 -1.33647548e+00 + -3.41357732e+00 -8.47048920e-01 + -3.06702448e-01 -6.64280634e-01 + -2.86930714e-01 -1.35268264e-01 + -3.15835557e+00 -5.43439253e-01 + 2.49541440e-01 -4.71733570e-01 + 2.71933912e+00 4.13308399e-01 + -2.43787038e+00 -1.08050547e+00 + -4.90234490e-01 -6.64069865e-01 + 8.99524451e-02 5.76180541e-01 + 5.00500404e+00 2.12125521e+00 + -1.73107940e-01 -2.28506575e-02 + 5.44938858e-01 -1.29523352e-01 + 5.13526842e+00 1.68785993e+00 + 1.70228304e+00 1.02601138e+00 + 3.58957507e+00 1.54396196e+00 + 1.85615738e+00 4.92916197e-01 + 2.55772147e+00 7.88438908e-01 + -1.57008279e+00 -4.17377300e-01 + -1.42548604e+00 -3.63684860e-01 + -8.52026118e-01 2.72052686e-01 + -5.10563077e+00 -2.35665994e+00 + -2.95517031e+00 -1.84945297e+00 + -2.91947959e+00 -1.66016784e+00 + -4.21462387e+00 -1.41131535e+00 + 6.59901121e-01 4.87156314e-01 + -9.75352532e-01 -4.50231285e-01 + -5.94084444e-01 -1.16922670e+00 + 7.50554615e-01 -9.83692552e-01 + 1.07054926e+00 2.77143030e-01 + -3.88079578e-01 -4.17737309e-02 + -9.59373733e-01 -8.85454886e-01 + -7.53560665e-02 -5.16223870e-02 + 9.84108158e-01 -5.89290700e-02 + 1.87272961e-01 -4.34238391e-01 + 6.86509981e-01 -3.15116460e-01 + -1.07762538e+00 6.58984161e-02 + 6.09266592e-01 6.91808473e-02 + -8.30529954e-01 -7.00454791e-01 + -9.13179464e-01 -6.31712891e-01 + 7.68744851e-01 1.09840676e+00 + -1.07606690e+00 -8.78390282e-01 + -1.71038184e+00 -5.73606033e-01 + 8.75982765e-01 3.66343143e-01 + -7.04919009e-01 -8.49182590e-01 + -1.00274668e+00 -7.99573611e-01 + -1.05562848e+00 -5.84060076e-01 + 4.03490015e+00 1.28679206e+00 + -3.53484804e+00 -1.71381255e+00 + 2.31527363e-01 1.04179397e-01 + -3.58592392e-02 3.74895739e-01 + 3.92253428e+00 1.81852726e+00 + -7.27384249e-01 -6.45605128e-01 + 4.65678097e+00 2.41379899e+00 + 1.16750534e+00 7.60718205e-01 + 1.15677059e+00 7.96225550e-01 + -1.42920261e+00 -4.66946295e-01 + 3.71148192e+00 1.88060191e+00 + 2.44052407e+00 3.84472199e-01 + -1.64535035e+00 -8.94530036e-01 + -3.69608753e+00 -1.36402754e+00 + 2.24419208e+00 9.69744889e-01 + 2.54822427e+00 1.22613039e+00 + 3.77484909e-01 -5.98521878e-01 + -3.61521175e+00 -1.11123912e+00 + 3.28113127e+00 1.52551775e+00 + -3.51030902e+00 -1.53913980e+00 + -2.44874505e+00 -6.30246005e-01 + -3.42516153e-01 -5.07352665e-01 + 1.09110502e+00 6.36821628e-01 + -2.49434967e+00 -8.02827146e-01 + 1.41763139e+00 -3.46591820e-01 + 1.61108619e+00 5.93871102e-01 + 3.97371717e+00 1.35552499e+00 + -1.33437177e+00 -2.83908670e-01 + -1.41606483e+00 -1.76402601e-01 + 2.23945322e-01 -1.77157065e-01 + 2.60271569e+00 2.40778251e-01 + -2.82213895e-02 1.98255474e-01 + 4.20727940e+00 1.31490863e+00 + 3.36944889e+00 1.57566635e+00 + 3.53049396e+00 1.73579350e+00 + -1.29170202e+00 -1.64196290e+00 + 9.27295604e-01 9.98808036e-01 + 1.75321843e-01 -2.83267817e-01 + -2.19069578e+00 -1.12814358e+00 + 1.66606031e+00 7.68006933e-01 + -7.13826035e-01 5.20881684e-02 + -3.43821888e+00 -2.36137021e+00 + -5.93210310e-01 1.21843813e-01 + -4.09800822e+00 -1.39893953e+00 + 2.74110954e+00 1.52728606e+00 + 1.72652512e+00 -1.25435113e-01 + 1.97722357e+00 6.40667481e-01 + 4.18635780e-01 3.57018509e-01 + -1.78303569e+00 -2.11864764e-01 + -3.52809366e+00 -2.58794450e-01 + -4.72407090e+00 -1.63870734e+00 + 1.73917807e+00 8.73251829e-01 + 4.37979356e-01 8.49210569e-01 + 3.93791881e+00 1.76269490e+00 + 2.79065411e+00 1.04019042e+00 + -8.47426142e-01 -3.40136892e-01 + -4.24389181e+00 -1.80253120e+00 + -1.86675870e+00 -7.64558265e-01 + 9.46212675e-01 -7.77681445e-02 + -2.82448462e+00 -1.33592449e+00 + -2.57938567e+00 -1.56554690e+00 + -2.71615767e+00 -6.27667233e-01 + -1.55999166e+00 -5.81013466e-01 + -4.24696864e-01 -7.44673250e-01 + 1.67592970e+00 7.68164292e-01 + 8.48455216e-01 -6.05681126e-01 + 6.12575454e+00 1.65607584e+00 + 1.38207327e+00 2.39261863e-01 + 3.13364450e+00 1.17154698e+00 + 1.71694858e+00 1.26744905e+00 + -1.61746367e+00 -8.80098073e-01 + -8.52196756e-01 -9.27299728e-01 + -1.51562462e-01 -8.36552490e-02 + -7.04792753e-01 -1.24726713e-02 + -3.35265757e+00 -1.82176312e+00 + 3.32173170e-01 -1.33405580e-01 + 4.95841013e-01 4.58292712e-01 + 1.57713955e+00 7.79272991e-01 + 2.09743109e+00 9.23542557e-01 + 3.90450311e-03 -8.42873164e-01 + 2.59519038e+00 7.56479591e-01 + -5.77643976e-01 -2.36401904e-01 + -5.22310654e-01 1.34187830e-01 + -2.22096086e+00 -7.75507719e-01 + 1.35907831e+00 7.80197510e-01 + 3.80355868e+00 1.16983476e+00 + 3.82746596e+00 1.31417718e+00 + 3.30451183e+00 1.55398159e+00 + -3.42917814e-01 -8.62281222e-02 + -2.59093020e+00 -9.29883526e-01 + 1.40928562e+00 1.08398346e+00 + 1.54400137e-01 3.35881092e-01 + 1.59171586e+00 1.18855802e+00 + -5.25164002e-01 -1.03104220e-01 + 2.20067959e+00 1.37074713e+00 + 6.97860830e-01 6.27718548e-01 + -4.59743507e-01 1.36061163e-01 + -1.04691963e-01 -2.16271727e-01 + -1.08905573e+00 -5.95510769e-01 + -1.00826983e+00 -5.38509162e-02 + -3.16402719e+00 -1.33414216e+00 + 1.47870874e-01 1.75234619e-01 + -2.57078234e-01 7.03316889e-02 + 1.81073945e+00 4.26901462e-01 + 2.65476530e+00 6.74217273e-01 + 1.27539811e+00 6.22914081e-01 + -3.76750499e-01 -1.20629449e+00 + 1.00177595e+00 -1.40660091e-01 + -2.98919265e+00 -1.65145013e+00 + -2.21557682e+00 -8.11123452e-01 + -3.22635378e+00 -1.65639056e+00 + -2.72868553e+00 -1.02812087e+00 + 1.26042797e+00 8.49005248e-01 + -9.38318534e-01 -9.87588651e-01 + 3.38013194e-01 -1.00237461e-01 + 1.91175691e+00 8.48716369e-01 + 4.30244344e-01 6.05539915e-02 + 2.21783435e+00 3.03268204e-01 + 1.78019576e+00 1.27377108e+00 + 1.59733274e+00 4.40674687e-02 + 3.97428484e+00 2.20881566e+00 + -2.41108677e+00 -6.01410418e-01 + -2.50796499e+00 -5.71169866e-01 + -3.71957427e+00 -1.38195726e+00 + -1.57992670e+00 1.32068593e-01 + -1.35278851e+00 -6.39349270e-01 + 1.23075932e+00 2.40445409e-01 + 1.35606530e+00 4.33180078e-01 + 9.60968518e-02 2.26734255e-01 + 6.22975063e-01 5.03431915e-02 + -1.47624851e+00 -3.60568238e-01 + -2.49337808e+00 -1.15083052e+00 + 2.15717792e+00 1.03071559e+00 + -3.07814376e-02 1.38700314e-02 + 4.52049499e-02 -4.86409775e-01 + 2.58231061e+00 1.14327809e-01 + 1.10999138e+00 -5.18568405e-01 + -2.19426443e-01 -5.37505538e-01 + -4.44740298e-01 6.78099955e-01 + 4.03379080e+00 1.49825720e+00 + -5.13182408e-01 -4.90201950e-01 + -6.90139716e-01 1.63875126e-01 + -8.17281461e-01 2.32155064e-01 + -2.92357619e-01 -8.02573544e-01 + -1.80769841e+00 -7.58907326e-01 + 2.16981590e+00 1.06728873e+00 + 1.98995203e-01 -6.84176682e-02 + -2.39546753e+00 -2.92873789e-01 + -4.24251021e+00 -1.46255564e+00 + -5.01411291e-01 -5.95712813e-03 + 2.68085809e+00 1.42883780e+00 + -4.13289873e+00 -1.62729388e+00 + 1.87957843e+00 3.63341638e-01 + -1.15270744e+00 -3.03563774e-01 + -4.43994248e+00 -2.97323905e+00 + -7.17067733e-01 -7.08349542e-01 + -3.28870393e+00 -1.19263863e+00 + -7.55325944e-01 -5.12703329e-01 + -2.07291938e+00 -2.65025085e-01 + -7.50073814e-01 -1.70771041e-01 + -8.77381404e-01 -5.47417325e-01 + -5.33725862e-01 5.15837119e-01 + 8.45056431e-01 2.82125560e-01 + -1.59598637e+00 -1.38743235e+00 + 1.41362902e+00 1.06407789e+00 + 1.02584504e+00 -3.68219466e-01 + -1.04644488e+00 -1.48769392e-01 + 2.66990191e+00 8.57633492e-01 + -1.84251857e+00 -9.82430175e-01 + 9.71404204e-01 -2.81934209e-01 + -2.50177989e+00 -9.21260335e-01 + -1.31060074e+00 -5.84488113e-01 + -2.12129400e-01 -3.06244708e-02 + -5.28933882e+00 -2.50663129e+00 + 1.90220541e+00 1.08662918e+00 + -3.99366086e-02 -6.87178973e-01 + -4.93417342e-01 4.37354182e-01 + 2.13494486e+00 1.37679569e+00 + 2.18396765e+00 5.81023868e-01 + -3.07866587e+00 -1.45384974e+00 + 6.10894119e-01 -4.17050124e-01 + -1.88766952e+00 -8.86160058e-01 + 3.34527253e+00 1.78571260e+00 + 6.87769059e-01 -5.01157336e-01 + 2.60470837e+00 1.45853560e+00 + -6.49315691e-01 -9.16112805e-01 + -1.29817687e+00 -2.15924339e-01 + -1.20100409e-03 -4.03137422e-01 + -1.36471594e+00 -6.93266356e-01 + 1.38682062e+00 7.15131598e-01 + 2.47830103e+00 1.24862305e+00 + -2.78288147e+00 -1.03329235e+00 + -7.33443403e-01 -6.11041652e-01 + -4.12745671e-01 -5.96133390e-02 + -2.58632336e+00 -4.51557058e-01 + -1.16570367e+00 -1.27065510e+00 + 2.76187104e+00 2.21895451e-01 + -3.80443767e+00 -1.66319902e+00 + 9.84658633e-01 6.81475569e-01 + 9.33814584e-01 -4.89335563e-02 + -4.63427997e-01 1.72989539e-01 + 1.82401546e+00 3.60164021e-01 + -5.36521077e-01 -8.08691351e-01 + -1.37367030e+00 -1.02126160e+00 + -3.70310682e+00 -1.19840844e+00 + -1.51894242e+00 -3.89510223e-01 + -3.67347940e-01 -3.25540516e-02 + -1.00988595e+00 1.82802194e-01 + 2.01622795e+00 7.86367901e-01 + 1.02440231e+00 8.79780360e-01 + -3.05971480e+00 -8.40901527e-01 + 2.73909457e+00 1.20558628e+00 + 2.39559056e+00 1.10786694e+00 + 1.65471544e+00 7.33824651e-01 + 2.18546787e+00 6.41168955e-01 + 1.47152266e+00 3.91839132e-01 + 1.45811155e+00 5.21820495e-01 + -4.27531469e-02 -3.52343068e-03 + -9.54948010e-01 -1.52313876e-01 + 7.57151215e-01 -5.68728854e-03 + -8.46205751e-01 -7.54580229e-01 + 4.14493548e+00 1.45532780e+00 + 4.58688968e-01 -4.54012803e-02 + -1.49295381e+00 -4.57471758e-01 + 1.80020351e+00 8.13724973e-01 + -5.82727738e+00 -2.18269581e+00 + -2.09017809e+00 -1.18305177e+00 + -2.31628303e+00 -7.21600235e-01 + -8.09679091e-01 -1.49101752e-01 + 8.88005605e-01 8.57940857e-01 + -1.44148219e+00 -3.10926299e-01 + 3.68828186e-01 -3.08848059e-01 + -6.63267389e-01 -8.58950139e-02 + -1.14702569e+00 -6.32147854e-01 + -1.51741715e+00 -8.53330564e-01 + -1.33903718e+00 -1.45875547e-01 + 4.12485387e+00 1.85620435e+00 + -2.42353639e+00 -2.92669850e-01 + 1.88708583e+00 9.35984730e-01 + 2.15585179e+00 6.30469051e-01 + -1.13627973e-01 -1.62554045e-01 + 2.04540494e+00 1.36599834e+00 + 2.81591381e+00 1.60897941e+00 + 3.02736260e-02 3.83255815e-03 + 7.97634013e-02 -2.82035099e-01 + -3.24607473e-01 -5.30065956e-01 + -3.91862894e+00 -1.94083334e+00 + 1.56360901e+00 7.93882743e-01 + -1.03905772e+00 6.25590229e-01 + 2.54746492e+00 1.64233560e+00 + -4.80774423e-01 -8.92298032e-02 + 9.06979990e-02 1.05020427e+00 + -2.47521290e+00 -1.78275982e-01 + -3.91871729e-01 3.80285423e-01 + 1.00658382e+00 4.58947483e-01 + 4.68102941e-01 1.02992741e+00 + 4.44242568e-01 2.89870239e-01 + 3.29684452e+00 1.44677474e+00 + -2.24983007e+00 -9.65574499e-01 + -3.54453926e-01 -3.99020325e-01 + -3.87429665e+00 -1.90079739e+00 + 2.02656674e+00 1.12444894e+00 + 3.77011621e+00 1.43200852e+00 + 1.61259275e+00 4.65417399e-01 + 2.28725434e+00 6.79181395e-01 + 2.75421009e+00 2.27327345e+00 + -2.40894409e+00 -1.03926359e+00 + 1.52996651e-01 -2.73373046e-02 + -2.63218977e+00 -7.22802821e-01 + 2.77688169e+00 1.15310186e+00 + 1.18832341e+00 4.73457165e-01 + -2.35536326e+00 -1.08034554e+00 + -5.84221627e-01 1.03505984e-02 + 2.96730300e+00 1.33478306e+00 + -8.61947692e-01 6.09137051e-02 + 8.22343921e-01 -8.14155286e-02 + 1.75809015e+00 1.07921470e+00 + 1.19501279e+00 1.05309972e+00 + -1.75901792e+00 9.75320161e-02 + 1.64398635e+00 9.54384323e-01 + -2.21878052e-01 -3.64847144e-01 + -2.03128968e+00 -8.57866419e-01 + 1.86750633e+00 7.08524487e-01 + 8.03972976e-01 3.47404314e-01 + 3.41203749e+00 1.39810900e+00 + 4.22397681e-01 -6.41440488e-01 + -4.88493360e+00 -1.58967816e+00 + -1.67649284e-01 -1.08485915e-01 + 2.11489023e+00 1.50506158e+00 + -1.81639929e+00 -3.85542192e-01 + 2.24044819e-01 -1.45100577e-01 + -3.39262411e+00 -1.44394324e+00 + 1.68706599e+00 2.29199618e-01 + -1.94093257e+00 -1.65975814e-01 + 8.28143367e-01 5.92109281e-01 + -8.29587998e-01 -9.57130831e-01 + -1.50011401e+00 -8.36802092e-01 + 2.40770449e+00 9.32820177e-01 + 7.41391309e-02 3.12878473e-01 + 1.87745264e-01 6.19231425e-01 + 9.57622692e-01 -2.20640033e-01 + 3.18479243e+00 1.02986233e+00 + 2.43133846e+00 8.41302677e-01 + -7.09963834e-01 1.99718943e-01 + -2.88253498e-01 -3.62772094e-01 + 5.14052574e+00 1.79304595e+00 + -3.27930993e+00 -1.29177973e+00 + -1.16723536e+00 1.29519656e-01 + 1.04801056e+00 3.41508300e-01 + -3.99256195e+00 -2.51176471e+00 + -7.62824318e-01 -6.84242153e-01 + 2.71524986e-02 5.35157164e-02 + 3.26430102e+00 1.34887262e+00 + -1.72357766e+00 -4.94524388e-01 + -3.81149536e+00 -1.28121944e+00 + 3.36919354e+00 1.10672075e+00 + -3.14841757e+00 -7.10713767e-01 + -3.16463676e+00 -7.58558435e-01 + -2.44745969e+00 -1.08816514e+00 + 2.79173264e-01 -2.19652051e-02 + 4.15309883e-01 6.07502790e-01 + -9.51007417e-01 -5.83976336e-01 + -1.47929839e+00 -8.39850409e-01 + 2.38335703e+00 6.16055149e-01 + -7.47749031e-01 -5.56164928e-01 + -3.65643622e-01 -5.06684411e-01 + -1.76634163e+00 -7.86382097e-01 + 6.76372222e-01 -3.06592181e-01 + -1.33505058e+00 -1.18301441e-01 + 3.59660179e+00 2.00424178e+00 + -7.88912762e-02 8.71956146e-02 + 1.22656397e+00 1.18149583e+00 + 4.24919729e+00 1.20082355e+00 + 2.94607456e+00 1.00676505e+00 + 7.46061275e-02 4.41761753e-02 + -2.47738025e-02 1.92737701e-01 + -2.20509316e-01 -3.79163193e-01 + -3.50222190e-01 3.58727299e-01 + -3.64788014e+00 -1.36107312e+00 + 3.56062799e+00 9.27032742e-01 + 1.04317289e+00 6.08035970e-01 + 4.06718718e-01 3.00628051e-01 + 4.33158086e+00 2.25860714e+00 + 2.13917145e-01 -1.72757967e-01 + -1.40637998e+00 -1.14119465e+00 + 3.61554872e+00 1.87797348e+00 + 1.01726871e+00 5.70255097e-01 + -7.04902551e-01 2.16444147e-01 + -2.51492186e+00 -8.52997369e-01 + 1.85097530e+00 1.15124496e+00 + -8.67569714e-01 -3.05682432e-01 + 8.07550858e-01 5.88901608e-01 + 1.85186755e-01 -1.94589367e-01 + -1.23378238e+00 -7.84128347e-01 + -1.22713161e+00 -4.21218235e-01 + 2.97751165e-01 2.81055275e-01 + 4.77703554e+00 1.66265524e+00 + 2.51549669e+00 7.49980674e-01 + 2.76510822e-01 1.40456909e-01 + 1.98740905e+00 -1.79608212e-01 + 9.35429145e-01 8.44344180e-01 + -1.20854492e+00 -5.00598453e-01 + 2.29936219e+00 8.10236668e-01 + 6.92555544e-01 -2.65891331e-01 + -1.58050994e+00 2.31237821e-01 + -1.50864880e+00 -9.49661690e-01 + -1.27689206e+00 -7.18260016e-01 + -3.12517127e+00 -1.75587113e+00 + 8.16062912e-02 -6.56551804e-01 + -5.02479939e-01 -4.67162543e-01 + -5.47435788e+00 -2.47799576e+00 + 1.95872901e-02 5.80874076e-01 + -1.59064958e+00 -6.34554756e-01 + -3.77521478e+00 -1.74301790e+00 + 5.89628224e-01 8.55736553e-01 + -1.81903543e+00 -7.50011008e-01 + 1.38557775e+00 3.71490991e-01 + 9.70032652e-01 -7.11356016e-01 + 2.63539625e-01 -4.20994771e-01 + 2.12154222e+00 8.19081400e-01 + -6.56977937e-01 -1.37810098e-01 + 8.91309581e-01 2.77864361e-01 + -7.43693195e-01 -1.46293770e-01 + 2.24447769e+00 4.00911438e-01 + -2.25169262e-01 2.04148801e-02 + 1.68744684e+00 9.47573007e-01 + 2.73086373e-01 3.30877195e-01 + 5.54294414e+00 2.14198009e+00 + -8.49238733e-01 3.65603298e-02 + 2.39685712e+00 1.17951039e+00 + -2.58230528e+00 -5.52116673e-01 + 2.79785277e+00 2.88833717e-01 + -1.96576188e-01 1.11652123e+00 + -4.69383301e-01 1.96496282e-01 + -1.95011845e+00 -6.15235169e-01 + 1.03379890e-02 2.33701239e-01 + 4.18933607e-01 2.77939814e-01 + -1.18473337e+00 -4.10051126e-01 + -7.61499744e-01 -1.43658094e+00 + -1.65586092e+00 -3.41615303e-01 + -5.58523700e-02 -5.21837080e-01 + -2.40331088e+00 -2.64521583e-01 + 2.24925206e+00 6.79843335e-02 + 1.46360479e+00 1.04271443e+00 + -3.09255443e+00 -1.82548953e+00 + 2.11325841e+00 1.14996627e+00 + -8.70657797e-01 1.02461839e-01 + -5.71056521e-01 9.71232588e-02 + -3.37870752e+00 -1.54091877e+00 + 1.03907189e+00 -1.35661392e-01 + 8.40057486e-01 6.12172413e-02 + -1.30998234e+00 -1.34077226e+00 + 7.53744974e-01 1.49447350e-01 + 9.13995056e-01 -1.81227962e-01 + 2.28386229e-01 3.74498520e-01 + 2.54829151e-01 -2.88802704e-01 + 1.61709009e+00 2.09319193e-01 + -1.12579380e+00 -5.95955338e-01 + -2.69610726e+00 -2.76222736e-01 + -2.63773329e+00 -7.84491970e-01 + -2.62167427e+00 -1.54792874e+00 + -4.80639856e-01 -1.30582102e-01 + -1.26130891e+00 -8.86841840e-01 + -1.24951950e+00 -1.18182622e+00 + -1.40107574e+00 -9.13695575e-01 + 4.99872179e-01 4.69014702e-01 + -2.03550193e-02 -1.48859738e-01 + -1.50189069e+00 -2.97714278e-02 + -2.07846113e+00 -7.29937809e-01 + -5.50576792e-01 -7.03151525e-01 + -3.88069238e+00 -1.63215295e+00 + 2.97032988e+00 6.43571144e-01 + -1.85999273e-01 1.18107620e+00 + 1.79249709e+00 6.65356160e-01 + 2.68842472e+00 1.35703255e+00 + 1.07675417e+00 1.39845588e-01 + 8.01226349e-01 2.11392275e-01 + 9.64329379e-01 3.96146195e-01 + -8.22529511e-01 1.96080831e-01 + 1.92481841e+00 4.62985744e-01 + 3.69756927e-01 3.77135799e-01 + 1.19807835e+00 8.87715050e-01 + -1.01363587e+00 -2.48151636e-01 + 8.53071010e-01 4.96887868e-01 + -3.41120553e+00 -1.35401843e+00 + -2.64787381e+00 -1.08690563e+00 + -1.11416759e+00 -4.43848915e-01 + 1.46242648e+00 6.17106076e-02 + -7.52968881e-01 -9.20972209e-01 + -1.22492228e+00 -5.40327617e-01 + 1.08001827e+00 5.29593785e-01 + -2.58706464e-01 1.13022085e-01 + -4.27394011e-01 1.17864354e-02 + -3.20728413e+00 -1.71224737e-01 + 1.71398530e+00 8.68885893e-01 + 2.12067866e+00 1.45092772e+00 + 4.32782616e-01 -3.34117769e-01 + 7.80084374e-01 -1.35100217e-01 + -2.05547729e+00 -4.70217750e-01 + 2.38379736e+00 1.09186058e+00 + -2.80825477e+00 -1.03320187e+00 + 2.63434576e+00 1.15671733e+00 + -1.60936214e+00 1.91843035e-01 + -5.02298769e+00 -2.32820708e+00 + 1.90349195e+00 1.45215416e+00 + 3.00232888e-01 3.24412586e-01 + -2.46503943e+00 -1.19550010e+00 + 1.06304233e+00 2.20136246e-01 + -2.99101388e+00 -1.58299318e+00 + 2.30071719e+00 1.12881362e+00 + -2.37587247e+00 -8.08298336e-01 + 7.27006308e-01 3.80828984e-01 + 2.61199061e+00 1.56473491e+00 + 8.33936357e-01 -1.42189425e-01 + 3.13291605e+00 1.77771210e+00 + 2.21917371e+00 5.68427075e-01 + 2.38867649e+00 9.06637262e-01 + -6.92959466e+00 -3.57682881e+00 + 2.57904824e+00 5.93959108e-01 + 2.71452670e+00 1.34436199e+00 + 4.39988761e+00 2.13124672e+00 + 5.71783077e-01 5.08346173e-01 + -3.65399429e+00 -1.18192861e+00 + 4.46176453e-01 3.75685594e-02 + -2.97501495e+00 -1.69459236e+00 + 1.60855728e+00 9.20930014e-01 + -1.44270290e+00 -1.93922306e-01 + 1.67624229e+00 1.66233866e+00 + -1.42579598e+00 -1.44990145e-01 + 1.19923176e+00 4.58490278e-01 + -9.00068460e-01 5.09701825e-02 + -1.69391694e+00 -7.60070300e-01 + -1.36576440e+00 -5.24244256e-01 + -1.03016748e+00 -3.44625878e-01 + 2.40519313e+00 1.09947587e+00 + 1.50365433e+00 1.06464802e+00 + -1.07609727e+00 -3.68897187e-01 + 2.44969069e+00 1.28486192e+00 + -1.25610307e+00 -1.14644789e+00 + 2.05962899e+00 4.31162369e-01 + -7.15886908e-01 -6.11587804e-02 + -6.92354119e-01 -7.85019920e-01 + -1.63016508e+00 -5.96944975e-01 + 1.90352536e+00 1.28197457e+00 + -4.01535243e+00 -1.81934488e+00 + -1.07534435e+00 -2.10544784e-01 + 3.25500866e-01 7.69603661e-01 + 2.18443365e+00 6.59773335e-01 + 8.80856790e-01 6.39505913e-01 + -2.23956372e-01 -4.65940132e-01 + -1.06766519e+00 -5.38388505e-03 + 7.25556863e-01 -2.91123488e-01 + -4.69451411e-01 7.89182650e-02 + 2.58146587e+00 1.29653243e+00 + 1.53747468e-01 7.69239075e-01 + -4.61152262e-01 -4.04151413e-01 + 1.48183517e+00 8.10079506e-01 + -1.83402614e+00 -1.36939322e+00 + 1.49315501e+00 7.95225425e-01 + 1.41922346e+00 1.05582774e-01 + 1.57473493e-01 9.70795657e-01 + -2.67603254e+00 -7.48562280e-01 + -8.49156216e-01 -6.05762529e-03 + 1.12944274e+00 3.67741591e-01 + 1.94228071e-01 5.28188141e-01 + -3.65610158e-01 4.05851838e-01 + -1.98839111e+00 -1.38452764e+00 + 2.73765752e+00 8.24150530e-01 + 7.63728641e-01 3.51617707e-01 + 5.78307267e+00 1.68103612e+00 + 2.27547227e+00 3.60876164e-01 + -3.50681697e+00 -1.74429984e+00 + 4.01241184e+00 1.26227829e+00 + 2.44946343e+00 9.06119057e-01 + -2.96638941e+00 -9.01532322e-01 + 1.11267643e+00 -3.43333381e-01 + -6.61868994e-01 -3.44666391e-01 + -8.34917179e-01 5.69478372e-01 + -1.91888454e+00 -3.03791075e-01 + 1.50397636e+00 8.31961240e-01 + 6.12260198e+00 2.16851807e+00 + 1.34093127e+00 8.86649385e-01 + 1.48748519e+00 8.26273697e-01 + 7.62243068e-01 2.64841396e-01 + -2.17604986e+00 -3.54219958e-01 + 2.64708640e-01 -4.38136718e-02 + 1.44725372e+00 1.18499914e-01 + -6.71259446e-01 -1.19526851e-01 + 2.40134595e-01 -8.90042323e-02 + -3.57238199e+00 -1.23166201e+00 + -3.77626645e+00 -1.19533443e+00 + -3.81101035e-01 -4.94160532e-01 + -3.02758757e+00 -1.18436066e+00 + 2.59116298e-01 1.38023047e+00 + 4.17900116e+00 1.12065959e+00 + 1.54598848e+00 2.89806755e-01 + 1.00656475e+00 1.76974511e-01 + -4.15730234e-01 -6.22681694e-01 + -6.00903565e-01 -1.43256959e-01 + -6.03652508e-01 -5.09936379e-01 + -1.94096658e+00 -9.48789544e-01 + -1.74464105e+00 -8.50491590e-01 + 1.17652544e+00 1.88118317e+00 + 2.35507776e+00 1.44000205e+00 + 2.63067924e+00 1.06692988e+00 + 2.88805386e+00 1.23924715e+00 + 8.27595008e-01 5.75364692e-01 + 3.91384216e-01 9.72781920e-02 + -1.03866816e+00 -1.37567768e+00 + -1.34777969e+00 -8.40266025e-02 + -4.12904508e+00 -1.67618340e+00 + 1.27918111e+00 3.52085961e-01 + 4.15361174e-01 6.28896189e-01 + -7.00539496e-01 4.80447955e-02 + -1.62332639e+00 -5.98236485e-01 + 1.45957300e+00 1.00305154e+00 + -3.06875603e+00 -1.25897545e+00 + -1.94708176e+00 4.85143006e-01 + 3.55744156e+00 -1.07468822e+00 + 1.21602223e+00 1.28768827e-01 + 1.89093098e+00 -4.70835659e-01 + -6.55759125e+00 2.70114082e+00 + 8.96843535e-01 -3.98115252e-01 + 4.13450429e+00 -2.32069236e+00 + 2.37764218e+00 -1.09098890e+00 + -1.11388901e+00 6.27083097e-01 + -6.34116929e-01 4.62816387e-01 + 2.90203079e+00 -1.33589143e+00 + 3.17457598e+00 -5.13575945e-01 + -1.76362299e+00 5.71820693e-01 + 1.66103362e+00 -8.99466249e-01 + -2.53947433e+00 8.40084780e-01 + 4.36631397e-01 7.24234261e-02 + -1.87589394e+00 5.08529113e-01 + 4.49563965e+00 -9.43365992e-01 + 1.78876299e+00 -1.27076149e+00 + -1.16269107e-01 -4.55078316e-01 + 1.92966079e+00 -8.05371385e-01 + 2.20632583e+00 -9.00919345e-01 + 1.52387824e+00 -4.82391996e-01 + 8.04004564e-01 -2.73650595e-01 + -7.75326067e-01 1.07469566e+00 + 1.83226282e+00 -4.52173344e-01 + 1.25079758e-01 -3.52895417e-02 + -9.90957437e-01 8.55993130e-01 + 1.71623322e+00 -7.08691667e-01 + -2.86175924e+00 6.75160955e-01 + -8.40817853e-01 -1.00361809e-01 + 1.33393000e+00 -4.65788123e-01 + 5.29394114e-01 -5.44881619e-02 + -8.07435599e-01 8.27353370e-01 + -4.33165824e+00 1.97299638e+00 + 1.26452422e+00 -8.34070486e-01 + 1.45996394e-02 2.97736043e-01 + -1.64489287e+00 6.72839598e-01 + -5.74234578e+00 3.20975117e+00 + 2.13841341e-02 3.64514015e-01 + 6.68084924e+00 -2.27464254e+00 + -3.22881590e+00 8.01879324e-01 + 3.02534313e-01 -4.56222796e-01 + -5.84520734e+00 1.95678162e+00 + 2.81515232e+00 -1.72101318e+00 + -2.39620908e-01 2.69145522e-01 + -7.41669691e-01 -2.30283281e-01 + -2.15682714e+00 3.45313021e-01 + 1.23475788e+00 -7.32276553e-01 + -1.71816113e-01 1.20419560e-02 + 1.89174235e+00 2.27435901e-01 + -3.64511114e-01 1.72260361e-02 + -3.24143860e+00 6.50125817e-01 + -2.25707409e+00 5.66970751e-01 + 1.03901456e+00 -1.00588433e+00 + -5.09159710e+00 1.58736109e+00 + 1.45534075e+00 -5.83787452e-01 + 4.28879587e+00 -1.58006866e+00 + 8.52384427e-01 -1.11042299e+00 + 4.51431615e+00 -2.63844265e+00 + -4.33042648e+00 1.86497078e+00 + -2.13568046e+00 5.82559743e-01 + -4.42568887e+00 1.26131214e+00 + 3.15821315e+00 -1.61515905e+00 + -3.14125204e+00 8.49604386e-01 + 6.54152300e-01 -2.04624711e-01 + -3.73374317e-01 9.94187820e-02 + -3.96177282e+00 1.27245623e+00 + 9.59825199e-01 -1.15547861e+00 + 3.56902055e+00 -1.46591091e+00 + 1.55433633e-02 6.93544345e-01 + 1.15684646e+00 -4.99836352e-01 + 3.11824573e+00 -4.75900506e-01 + -8.61706369e-01 -3.50774059e-01 + 9.89057391e-01 -7.16878802e-01 + -4.94787870e+00 2.09137481e+00 + 1.37777347e+00 -1.34946349e+00 + -1.13161577e+00 8.05114754e-01 + 8.12020675e-01 -1.04849421e+00 + 4.73783881e+00 -2.26718812e+00 + 8.99579366e-01 -8.89764451e-02 + 4.78524868e+00 -2.25795843e+00 + 1.75164590e+00 -1.73822209e-01 + 1.30204590e+00 -7.26724717e-01 + -7.26526403e-01 -5.23925361e-02 + 2.01255351e+00 -1.69965366e+00 + 9.87852740e-01 -4.63577220e-01 + 2.45957762e+00 -1.29278962e+00 + -3.13817948e+00 1.64433038e+00 + -1.76302159e+00 9.62784302e-01 + -1.91106331e+00 5.81460008e-01 + -3.30883001e+00 1.30378978e+00 + 5.54376450e-01 3.78814272e-01 + 1.09982111e+00 -1.47969612e+00 + -2.61300705e-02 -1.42573464e-01 + -2.22096157e+00 7.75684440e-01 + 1.70319323e+00 -2.89738444e-01 + -1.43223842e+00 6.39284281e-01 + 2.34360959e-01 -1.64379268e-01 + -2.67147991e+00 9.46548086e-01 + 1.51131425e+00 -4.91594395e-01 + -2.48446856e+00 1.01286123e+00 + 1.50534658e-01 -2.94620246e-01 + -1.66966792e+00 1.67755508e+00 + -1.50094241e+00 3.30163095e-01 + 2.27681194e+00 -1.08064317e+00 + 2.05122965e+00 -1.15165939e+00 + -4.23509309e-01 -6.56906167e-02 + 1.80084023e+00 -1.07228556e+00 + -2.65769521e+00 1.18023206e+00 + 2.02852676e+00 -8.06793574e-02 + -4.49544185e+00 2.68200163e+00 + -7.50043216e-01 1.17079331e+00 + 6.80060893e-02 3.99055351e-01 + -3.83634635e+00 1.38406887e+00 + 3.24858545e-01 -9.25273218e-02 + -2.19895100e+00 1.47819500e+00 + -3.61569522e-01 -1.03188739e-01 + 1.12180375e-01 -9.52696354e-02 + -1.31477803e+00 1.79900570e-01 + 2.39573628e+00 -6.09739269e-01 + -1.00135700e+00 6.02837296e-01 + -4.11994589e+00 2.49599192e+00 + -1.54196236e-01 -4.84921951e-01 + 5.92569908e-01 -1.87310359e-01 + 3.85407741e+00 -1.50979925e+00 + 5.17802528e+00 -2.26032607e+00 + -1.37018916e+00 1.87111822e-01 + 8.46682996e-01 -3.56676331e-01 + -1.17559949e+00 5.29057734e-02 + -5.56475671e-02 6.79049243e-02 + 1.07851745e+00 -5.14535101e-01 + -2.71622446e+00 1.00151846e+00 + -1.08477208e+00 8.81391054e-01 + 5.50755824e-01 -5.20577727e-02 + 4.70885495e+00 -2.04220397e+00 + -1.87375336e-01 -6.16962830e-02 + 3.52097100e-01 2.21163550e-01 + 7.07929984e-01 -1.75827590e-01 + -1.22149219e+00 1.83084346e-01 + 2.58247412e+00 -6.15914898e-01 + -6.01206182e-01 -2.29832987e-01 + 9.83360449e-01 -3.75870060e-01 + -3.20027685e+00 1.35467480e+00 + 1.79178978e+00 -1.38531981e+00 + -3.30376867e-01 -1.16250192e-01 + -1.89053055e+00 5.68463567e-01 + -4.20604849e+00 1.65429681e+00 + -1.01185529e+00 1.92801240e-01 + -6.18819882e-01 5.42206996e-01 + -5.08091672e+00 2.61598591e+00 + -2.62570344e+00 2.51590658e+00 + 3.05577906e+00 -1.49090609e+00 + 2.77609677e+00 -1.37681378e+00 + -7.93515301e-02 4.28072744e-01 + -2.08359471e+00 8.94334295e-01 + 2.20163801e+00 4.01127167e-02 + -1.18145785e-01 -2.06822464e-01 + -2.74788298e-01 2.96250607e-01 + 1.59613555e+00 -3.87246203e-01 + -3.82971472e-01 -3.39716093e-02 + -4.20311307e-02 3.88529510e-01 + 1.52128574e+00 -9.33138876e-01 + -9.06584458e-01 -2.75016094e-02 + 3.56216834e+00 -9.99384622e-01 + 2.11964220e+00 -9.98749118e-02 + 4.01203480e+00 -2.03032745e+00 + -1.24171557e+00 1.97596725e-01 + -1.57230455e+00 4.14126609e-01 + -1.85484741e+00 5.40041563e-01 + 1.76329831e+00 -6.95967734e-01 + -2.29439232e-01 5.08669245e-01 + -5.45124276e+00 2.26907549e+00 + -5.71364288e-02 5.04476476e-01 + 3.12468018e+00 -1.46358879e+00 + 8.20017359e-01 6.51949028e-01 + -1.33977500e+00 2.83634232e-04 + -1.83311685e+00 1.23947117e+00 + 6.31205922e-01 1.19792164e-02 + -2.21967834e+00 6.94056232e-01 + -1.41693842e+00 9.93526233e-01 + -7.58885703e-01 6.78547347e-01 + 3.60239086e+00 -1.08644935e+00 + 6.72217073e-02 3.00036011e-02 + -3.42680958e-01 -3.48049352e-01 + 1.87546079e+00 -4.78018246e-01 + 7.00485821e-01 -3.52905383e-01 + -8.54580948e-01 8.17330861e-01 + 8.19123706e-01 -5.73927281e-01 + 2.70855639e-01 -3.08940052e-01 + -1.05059952e+00 3.27873168e-01 + 1.08282999e+00 4.84559349e-02 + -7.89899220e-01 1.22291138e+00 + -2.87939816e+00 7.17403497e-01 + -2.08429452e+00 8.87409226e-01 + 1.58409232e+00 -4.74123532e-01 + 1.26882735e+00 1.59162510e-01 + -2.53782993e+00 6.18253491e-01 + -8.92757445e-01 3.35979011e-01 + 1.31867900e+00 -1.17355054e+00 + 1.14918879e-01 -5.35184038e-01 + -1.70288738e-01 5.35868087e-02 + 4.21355121e-01 5.41848690e-02 + 2.07926943e+00 -5.72538144e-01 + 4.08788970e-01 3.77655777e-01 + -3.39631381e+00 9.84216764e-01 + 2.94170163e+00 -1.83120916e+00 + -7.94798752e-01 7.39889052e-01 + 1.46555463e+00 -4.62275563e-01 + 2.57255955e+00 -1.04671434e+00 + 8.45042540e-01 -1.96952892e-01 + -3.23526646e+00 1.60049846e+00 + 3.21948565e+00 -8.88376674e-01 + 1.43005104e+00 -9.21561086e-01 + 8.82360506e-01 2.98403872e-01 + -8.91168097e-01 1.01319072e+00 + -5.13215241e-01 -2.47182649e-01 + -1.35759444e+00 7.07450608e-02 + -4.04550983e+00 2.23534867e+00 + 1.39348883e+00 3.81637747e-01 + -2.85676418e+00 1.53240862e+00 + -1.37183120e+00 6.37977425e-02 + -3.88195859e+00 1.73887145e+00 + 1.19509776e+00 -6.25013512e-01 + -2.80062734e+00 1.79840585e+00 + 1.96558429e+00 -4.70997234e-01 + 1.93111352e+00 -9.70318441e-01 + 3.57991190e+00 -1.65065116e+00 + 2.12831714e+00 -1.11531708e+00 + -3.95661018e-01 -8.54339904e-02 + -2.41630441e+00 1.65166304e+00 + 7.55412624e-01 -1.53453579e-01 + -1.77043450e+00 1.39928715e+00 + -9.32631260e-01 8.73649199e-01 + 1.53342205e+00 -8.39569765e-01 + -6.29846924e-02 1.25023084e-01 + 3.31509049e+00 -1.10733235e+00 + -2.18957109e+00 3.07376993e-01 + -2.35740747e+00 6.47437564e-01 + -2.22142438e+00 8.47318938e-01 + -6.51401147e-01 3.48398562e-01 + 2.75763095e+00 -1.21390708e+00 + 1.12550484e+00 -5.61412847e-01 + -5.65053161e-01 6.74365205e-02 + 1.68952456e+00 -6.57566096e-01 + 8.95598401e-01 3.96738993e-01 + -1.86537066e+00 9.44129208e-01 + -2.59933294e+00 2.57423247e-01 + -6.59598267e-01 1.91828851e-02 + -2.64506676e+00 8.41783205e-01 + -1.25911802e+00 5.52425066e-01 + -1.39754507e+00 3.73689222e-01 + 5.49550729e-02 1.35071215e+00 + 3.31874811e+00 -1.05682424e+00 + 3.63159604e+00 -1.42864695e+00 + -4.45944617e+00 1.42889446e+00 + 5.87314342e-01 -4.88892988e-01 + -7.26130820e-01 1.51936106e-01 + -1.79246441e+00 6.05888105e-01 + -5.50948207e-01 6.21443081e-01 + -3.17246063e-01 1.77213880e-01 + -2.00098937e+00 1.23799074e+00 + 4.33790961e+00 -1.08490465e+00 + -2.03114114e+00 1.31613237e+00 + -6.29216542e+00 1.92406317e+00 + -1.60265624e+00 8.87947500e-01 + 8.64465062e-01 -8.37416270e-01 + -2.14273937e+00 8.05485900e-01 + -2.36844256e+00 6.17915124e-01 + -1.40429636e+00 6.78296866e-01 + 9.99019988e-01 -5.84297572e-01 + 7.38824546e-01 1.68838678e-01 + 1.45681238e+00 3.04641461e-01 + 2.15914949e+00 -3.43089227e-01 + -1.23895930e+00 1.05339864e-01 + -1.23162264e+00 6.46629863e-01 + 2.28183862e+00 -9.24157063e-01 + -4.29615882e-01 5.69130863e-01 + -1.37449121e+00 -9.12032183e-01 + -7.33890904e-01 -3.91865471e-02 + 8.41400661e-01 -4.76002200e-01 + -1.73349274e-01 -6.84143467e-02 + 3.16042891e+00 -1.32651856e+00 + -3.78244609e+00 2.38619718e+00 + -3.69634380e+00 2.22368561e+00 + 1.83766344e+00 -1.65675953e+00 + -1.63206002e+00 1.19484469e+00 + 3.68480064e-01 -5.70764494e-01 + 3.61982479e-01 1.04274409e-01 + 2.48863048e+00 -1.13285542e+00 + -2.81896488e+00 9.47958768e-01 + 5.74952901e-01 -2.75959392e-01 + 3.72783275e-01 -3.48937848e-01 + 1.95935716e+00 -1.06750415e+00 + 5.19357531e+00 -2.32070803e+00 + 4.09246149e+00 -1.89976700e+00 + -3.36666087e-01 8.17645057e-02 + 1.85453493e-01 3.76913151e-01 + -3.06458262e+00 1.34106402e+00 + -3.13796566e+00 7.00485099e-01 + 1.42964058e+00 -1.35536932e-01 + -1.23440423e-01 4.60094177e-02 + -2.86753037e+00 -5.21724160e-02 + 2.67113726e+00 -1.83746924e+00 + -1.35335062e+00 1.28238073e+00 + -2.43569899e+00 1.25998539e+00 + 1.26036740e-01 -2.35416844e-01 + -1.35725745e+00 7.37788491e-01 + -3.80897538e-01 3.30757889e-01 + 6.58694434e-01 -1.07566603e+00 + 2.11273640e+00 -9.02260632e-01 + 4.00755057e-01 -2.49229150e-02 + -1.80095812e+00 9.73099742e-01 + -2.68408372e+00 1.63737364e+00 + -2.66079826e+00 7.47289412e-01 + -9.92321439e-02 -1.49331396e-01 + 4.45678251e+00 -1.80352394e+00 + 1.35962915e+00 -1.31554389e+00 + -7.76601417e-01 -9.66173523e-02 + 1.68096348e+00 -6.27235133e-01 + 1.53081227e-01 -3.54216830e-01 + -1.54913095e+00 3.43689269e-01 + 5.29187357e-02 -6.73916964e-01 + -2.06606084e+00 8.34784242e-01 + 1.73701179e+00 -6.06467340e-01 + 1.55856757e+00 -2.58642780e-01 + 1.04349101e+00 -4.43027348e-01 + -1.02397719e+00 1.01308824e+00 + -2.13860204e-01 -4.73347361e-01 + -2.59004955e+00 1.43367853e+00 + 7.98457679e-01 2.18621627e-02 + -1.32974762e+00 4.61802208e-01 + 3.21419359e-01 2.30723316e-02 + 2.87201888e-02 6.24566672e-02 + -1.22261418e+00 6.02340363e-01 + 1.28750335e+00 -3.34839548e-02 + -9.67952623e-01 4.34470505e-01 + 2.02850324e+00 -9.05160255e-01 + -4.13946010e+00 2.33779091e+00 + -4.47508806e-01 3.06440495e-01 + -3.91543394e+00 1.68251022e+00 + -6.45193001e-01 5.29781162e-01 + -2.15518916e-02 5.07278355e-01 + -2.83356868e+00 1.00670227e+00 + 1.82989749e+00 -1.37329222e+00 + -1.09330213e+00 1.08560688e+00 + 1.90533722e+00 -1.28905879e+00 + 2.33986084e+00 2.30642626e-02 + 8.01940220e-01 -1.63986962e+00 + -4.23415165e+00 2.07530423e+00 + 9.33382522e-01 -7.62917211e-01 + -1.84033954e+00 1.07469401e+00 + -2.81938669e+00 1.07342024e+00 + -7.05169988e-01 2.13124943e-01 + 5.09598137e-01 1.32725493e-01 + -2.34558226e+00 8.62383168e-01 + -1.70322072e+00 2.70893796e-01 + 1.23652660e+00 -7.53216034e-02 + 2.84660646e+00 -3.48178304e-02 + 2.50250128e+00 -1.27770855e+00 + -1.00279469e+00 8.77194218e-01 + -4.34674121e-02 -2.12091350e-01 + -5.84151289e-01 1.50382340e-01 + -1.79024013e+00 4.24972808e-01 + -1.23434666e+00 -8.85546570e-02 + 1.36575412e+00 -6.42639880e-01 + -1.98429947e+00 2.27650336e-01 + 2.36253589e+00 -1.51340773e+00 + 8.79157643e-01 6.84142159e-01 + -2.18577755e+00 2.76526200e-01 + -3.55473434e-01 8.29976561e-01 + 1.16442595e+00 -5.97699411e-01 + -7.35528097e-01 2.40318183e-01 + -1.73702631e-01 7.33788663e-02 + -1.40451745e+00 3.24899628e-01 + -2.05434385e+00 5.68123738e-01 + 8.47876642e-01 -5.74224294e-01 + -6.91955602e-01 1.26009087e+00 + 2.56574498e+00 -1.15602581e+00 + 3.93306545e+00 -1.38398209e+00 + -2.73230251e+00 4.89062581e-01 + -1.04315474e+00 6.06335547e-01 + 1.23231431e+00 -4.46675065e-01 + -3.93035285e+00 1.43287651e+00 + -1.02132111e+00 9.58919791e-01 + -1.49425352e+00 1.06456165e+00 + -6.26485337e-01 1.03791402e+00 + -6.61772998e-01 2.63275425e-01 + -1.80940386e+00 5.70767403e-01 + 9.83720450e-01 -1.39449756e-01 + -2.24619662e+00 9.01044870e-01 + 8.94343014e-01 5.31038678e-02 + 1.95518199e-01 -2.81343295e-01 + -2.30533019e-01 -1.74478106e-01 + -2.01550361e+00 5.55958010e-01 + -4.36281469e+00 1.94374226e+00 + -5.18530457e+00 2.89278357e+00 + 2.67289101e+00 -2.98511449e-01 + -1.53566179e+00 -1.00588944e-01 + -6.09943217e-02 -1.56986047e-01 + -5.22146452e+00 1.66209208e+00 + -3.69777478e+00 2.26154873e+00 + 2.24607181e-01 -4.86934960e-01 + 2.49909450e+00 -1.03033370e+00 + -1.07841120e+00 8.22388054e-01 + -3.20697089e+00 1.09536143e+00 + 3.43524232e+00 -1.47289362e+00 + -5.65784134e-01 4.60365175e-01 + -1.76714734e+00 1.57752346e-01 + -7.77620365e-01 5.60153443e-01 + 6.34399352e-01 -5.22339836e-01 + 2.91011875e+00 -9.72623380e-01 + -1.19286824e+00 6.32370253e-01 + -2.18327609e-01 8.23953181e-01 + 3.42430842e-01 1.37098055e-01 + 1.28658034e+00 -9.11357320e-01 + 2.06914465e+00 -6.67556382e-01 + -6.69451020e-01 -6.38605102e-01 + -2.09312398e+00 1.16743634e+00 + -3.63778357e+00 1.91919157e+00 + 8.74685911e-01 -1.09931208e+00 + -3.91496791e+00 1.00808357e+00 + 1.29621330e+00 -8.32239802e-01 + 9.00222045e-01 -1.31159793e+00 + -1.12242062e+00 1.98517079e-01 + -3.71932852e-01 1.31667093e-01 + -2.23829610e+00 1.26328346e+00 + -2.08365062e+00 9.93385336e-01 + -1.91082720e+00 7.45866855e-01 + 4.38024917e+00 -2.05901118e+00 + -2.28872886e+00 6.85279335e-01 + 1.01274497e-01 -3.26227153e-01 + -5.04447572e-01 -3.18619513e-01 + 1.28537006e+00 -1.04573551e+00 + -7.83175212e-01 1.54791645e-01 + -3.89239175e+00 1.60017929e+00 + -8.87877111e-01 -1.04968005e-01 + 9.32215179e-01 -5.58691113e-01 + -6.44977127e-01 -2.23018375e-01 + 1.10141900e+00 -1.00666432e+00 + 2.92755687e-01 -1.45480350e-01 + 7.73580681e-01 -2.21150567e-01 + -1.40873709e+00 7.61548044e-01 + -8.89031805e-01 -3.48542923e-01 + 4.16844267e-01 -2.39914494e-01 + -4.64265832e-01 7.29581138e-01 + 1.99835179e+00 -7.70542813e-01 + 4.20523191e-02 -2.18783563e-01 + -6.32611758e-01 -3.09926115e-01 + 6.82912198e-02 -8.48327050e-01 + 1.92425229e+00 -1.37876951e+00 + 3.49461782e+00 -1.88354255e+00 + -3.25209026e+00 1.49809395e+00 + 6.59273182e-01 -2.37435654e-01 + -1.15517300e+00 8.46134387e-01 + 1.26756151e+00 -4.58988026e-01 + -3.99178418e+00 2.04153008e+00 + 7.05687841e-01 -6.83433306e-01 + -1.61997342e+00 8.16577004e-01 + -3.89750399e-01 4.29753250e-01 + -2.53026432e-01 4.92861432e-01 + -3.16788324e+00 4.44285524e-01 + -7.86248901e-01 1.12753716e+00 + -3.02351433e+00 1.28419015e+00 + -1.30131355e+00 1.71226678e+00 + -4.08843475e+00 1.62063214e+00 + -3.09209403e+00 1.19958520e+00 + 1.49102271e+00 -1.11834864e+00 + -3.18059348e+00 5.74587042e-01 + 2.06054867e+00 3.25797860e-03 + -3.50999200e+00 2.02412428e+00 + -8.26610023e-01 3.46528211e-01 + 2.00546034e+00 -4.07333110e-01 + -9.69941653e-01 4.80953753e-01 + 4.47925660e+00 -2.33127314e+00 + 2.03845790e+00 -9.90439915e-01 + -1.11349191e+00 4.31183918e-01 + -4.03628396e+00 1.68509679e+00 + -1.48177601e+00 7.74322088e-01 + 3.07369385e+00 -9.57465886e-01 + 2.39011286e+00 -6.44506921e-01 + 2.91561991e+00 -8.78627328e-01 + 1.10212733e+00 -4.21637388e-01 + 5.31985231e-01 -6.17445696e-01 + -6.82340929e-01 -2.93529716e-01 + 1.94290679e+00 -4.64268634e-01 + 1.92262116e+00 -7.93142835e-01 + 4.73762800e+00 -1.63654174e+00 + -3.17848641e+00 8.05791391e-01 + 4.08739432e+00 -1.80816807e+00 + -7.60648826e-01 1.24216138e-01 + -2.24716400e+00 7.90020937e-01 + 1.64284052e+00 -7.18784070e-01 + 1.04410012e-01 -7.11195880e-02 + 2.18268225e+00 -7.01767831e-01 + 2.06218013e+00 -8.70251746e-01 + -1.35266581e+00 7.08456358e-01 + -1.38157779e+00 5.14401086e-01 + -3.28326008e+00 1.20988399e+00 + 8.85358917e-01 -8.12213495e-01 + -2.34067500e+00 3.67657353e-01 + 3.96878127e+00 -1.66841450e+00 + 1.36518053e+00 -8.33436812e-01 + 5.25771988e-01 -5.06121987e-01 + -2.25948361e+00 1.30663765e+00 + -2.57662070e+00 6.32114628e-01 + -3.43134685e+00 2.38106008e+00 + 2.31571924e+00 -1.56566818e+00 + -2.95397202e+00 1.05661888e+00 + -1.35331242e+00 6.76383411e-01 + 1.40977132e+00 -1.17775938e+00 + 1.52561996e+00 -9.83147176e-01 + 2.26550832e+00 -2.10464123e-02 + 6.23371684e-01 -5.30768122e-01 + -4.42356624e-01 9.72226986e-01 + 2.31517901e+00 -1.08468105e+00 + 1.97236640e+00 -1.42016619e+00 + 3.18618687e+00 -1.45056343e+00 + -2.75880360e+00 5.40254980e-01 + -1.92916581e+00 1.45029864e-01 + 1.90022524e+00 -6.03805754e-01 + -1.05446211e+00 5.74361752e-01 + 1.45990390e+00 -9.28233993e-01 + 5.14960557e+00 -2.07564096e+00 + -7.53104842e-01 1.55876958e-01 + 8.09490983e-02 -8.58886384e-02 + -1.56894969e+00 4.53497227e-01 + 1.36944658e-01 5.60670875e-01 + -5.32635329e-01 4.40309945e-01 + 1.32507853e+00 -5.83670099e-01 + 1.20676031e+00 -8.02296831e-01 + -3.65023422e+00 1.17211368e+00 + 1.53393850e+00 -6.17771312e-01 + -3.99977129e+00 1.71415137e+00 + 5.70705058e-01 -4.60771539e-01 + -2.20608002e+00 1.07866596e+00 + -1.09040244e+00 6.77441076e-01 + -5.09886482e-01 -1.97282128e-01 + -1.58062785e+00 6.18333697e-01 + -1.53295020e+00 4.02168701e-01 + -5.18580598e-01 2.25767177e-01 + 1.59514316e+00 -2.54983617e-01 + -5.91938655e+00 2.68223782e+00 + 2.84200509e+00 -1.04685313e+00 + 1.31298664e+00 -1.16672614e+00 + -2.36660033e+00 1.81359460e+00 + 6.94163290e-02 3.76658816e-01 + 2.33973934e+00 -8.33173023e-01 + -8.24640389e-01 7.83717285e-01 + -1.02888281e+00 1.04680766e+00 + 1.34750745e+00 -5.89568160e-01 + -2.48761231e+00 7.44199284e-01 + -1.04501559e+00 4.72326911e-01 + -3.14610089e+00 1.89843692e+00 + 2.13003416e-01 5.76633620e-01 + -1.69239608e+00 5.66070021e-01 + 1.80491280e+00 -9.31701080e-01 + -6.94362572e-02 6.96026587e-01 + 1.36502578e+00 -6.85599000e-02 + -7.76764337e-01 3.64328661e-01 + -2.67322167e+00 6.80150021e-01 + 1.84338485e+00 -1.18487494e+00 + 2.88009231e+00 -1.25700411e+00 + 1.17114433e+00 -7.69727080e-01 + 2.11576167e+00 2.81502116e-01 + -1.51470088e+00 2.61553540e-01 + 1.18923669e-01 -1.17890202e-01 + 4.48359786e+00 -1.81427466e+00 + -1.27055948e+00 9.92388998e-01 + -8.00276606e-01 9.11326621e-02 + 7.51764024e-01 -1.03676498e-01 + 1.35769348e-01 -2.11470084e-01 + 2.50731332e+00 -1.12418270e+00 + -2.49752781e-01 7.81224033e-02 + -6.23037902e-01 3.16599691e-01 + -3.93772902e+00 1.37195391e+00 + 1.74256361e+00 -1.12363582e+00 + -1.49737281e+00 5.98828310e-01 + 7.75592115e-01 -4.64733802e-01 + -2.26027693e+00 1.36991118e+00 + -1.62849836e+00 7.36899107e-01 + 2.36850751e+00 -9.32126872e-01 + 5.86169745e+00 -2.49342512e+00 + -5.37092226e-01 1.23821274e+00 + 2.80535867e+00 -1.93363302e+00 + -1.77638106e+00 9.10050276e-01 + 3.02692018e+00 -1.60774676e+00 + 1.97833084e+00 -1.50636531e+00 + 9.09168906e-01 -8.83799359e-01 + 2.39769655e+00 -7.56977869e-01 + 1.47283981e+00 -1.06749890e+00 + 2.92060943e-01 -6.07040605e-01 + -2.09278201e+00 7.71858590e-01 + 7.10015905e-01 -5.42768432e-01 + -2.16826169e-01 1.56897896e-01 + 4.56288247e+00 -2.08912680e+00 + -6.63374020e-01 6.67325183e-01 + 1.80564442e+00 -9.76366134e-01 + 3.28720168e+00 -4.66575145e-01 + -1.60463695e-01 -2.58428153e-01 + 1.78590750e+00 -3.96427146e-01 + 2.75950306e+00 -1.82102856e+00 + -1.18234310e+00 6.28073320e-01 + 4.11415835e+00 -2.33551216e+00 + 1.38721004e+00 -2.77450622e-01 + -2.94903545e+00 1.74813352e+00 + 8.67290400e-01 -6.51667894e-01 + 2.70022274e+00 -8.11832480e-01 + -2.06766146e+00 8.24047249e-01 + 3.90717142e+00 -1.20155758e+00 + -2.95102809e+00 1.36667968e+00 + 6.08815147e+00 -2.60737974e+00 + 2.78576476e+00 -7.86628755e-01 + -3.26258407e+00 1.09302450e+00 + 1.59849422e+00 -1.09705202e+00 + -2.50600710e-01 1.63243175e-01 + -4.90477087e-01 -4.57729572e-01 + -1.24837181e+00 3.22157840e-01 + -2.46341049e+00 1.06517849e+00 + 9.62880751e-01 4.56962496e-01 + 3.99964487e-01 2.07472802e-01 + 6.36657705e-01 -3.46400942e-02 + 4.91231407e-02 -1.40289235e-02 + -4.66683524e-02 -3.72326100e-01 + -5.22049702e-01 -1.70440260e-01 + 5.27062938e-01 -2.32628395e-01 + -2.69440318e+00 1.18914874e+00 + 3.65087539e+00 -1.53427267e+00 + -1.16546364e-01 4.93245392e-02 + 7.55931384e-01 -3.02980139e-01 + 2.06338745e+00 -6.24841225e-01 + 1.31177908e-01 7.29338183e-01 + 1.48021784e+00 -6.39509896e-01 + -5.98656707e-01 2.84525503e-01 + -2.18611080e+00 1.79549812e+00 + -2.91673624e+00 2.15772237e-01 + -8.95591350e-01 7.68250538e-01 + 1.36139762e+00 -1.93845144e-01 + 5.45730414e+00 -2.28114404e+00 + 3.22747247e-01 9.33582332e-01 + -1.46384504e+00 1.12801186e-01 + 4.26728166e-01 -2.33481242e-01 + -1.41327270e+00 8.16103740e-01 + -2.53998067e-01 1.44906646e-01 + -1.32436467e+00 1.87556361e-01 + -3.77313086e+00 1.32896038e+00 + 3.77651731e+00 -1.76548043e+00 + -2.45297093e+00 1.32571926e+00 + -6.55900588e-01 3.56921462e-01 + 9.25558722e-01 -4.51988954e-01 + 1.20732231e+00 -3.02821614e-01 + 3.72660154e-01 -1.89365208e-01 + -1.77090939e+00 9.18087975e-01 + 3.01127567e-01 2.67965829e-01 + -1.76708900e+00 4.62069259e-01 + -2.71812099e+00 1.57233508e+00 + -5.35297633e-01 4.99231535e-01 + 1.50507631e+00 -9.85763646e-01 + 3.00424787e+00 -1.29837562e+00 + -4.99311105e-01 3.91086482e-01 + 1.30125207e+00 -1.26247924e-01 + 4.01699483e-01 -4.46909391e-01 + -1.33635257e+00 5.12068703e-01 + 1.39229757e+00 -9.10974858e-01 + -1.74229508e+00 1.49475978e+00 + -1.21489414e+00 4.04193753e-01 + -3.36537605e-01 -6.74335427e-01 + -2.79186828e-01 8.48314720e-01 + -2.03080140e+00 1.66599815e+00 + -3.53064281e-01 -7.68582906e-04 + -5.30305657e+00 2.91091546e+00 + -1.20049972e+00 8.26578358e-01 + 2.95906989e-01 2.40215920e-01 + -1.42955534e+00 4.63480310e-01 + -1.87856619e+00 8.21459385e-01 + -2.71124720e+00 1.80246843e+00 + -3.06933780e+00 1.22235760e+00 + 5.21935582e-01 -1.27298218e+00 + -1.34175797e+00 7.69018937e-01 + -1.81962785e+00 1.15528991e+00 + -3.99227550e-01 2.93821598e-01 + 1.22533179e+00 -4.73846323e-01 + -2.08068359e-01 -1.75039817e-01 + -2.03068526e+00 1.50370503e+00 + -3.27606113e+00 1.74906330e+00 + -4.37802587e-01 -2.26956048e-01 + -7.69774213e-02 -3.54922468e-01 + 6.47160749e-02 -2.07334721e-01 + -1.37791524e+00 4.43766709e-01 + 3.29846803e+00 -1.04060799e+00 + -3.63704046e+00 1.05800226e+00 + -1.26716116e+00 1.13077353e+00 + 1.98549075e+00 -1.31864807e+00 + 1.85159500e+00 -5.78629560e-01 + -1.55295206e+00 1.23655857e+00 + 6.76026255e-01 9.18824125e-02 + 1.23418960e+00 -4.68162027e-01 + 2.43186642e+00 -9.22422440e-01 + -3.18729701e+00 1.77582673e+00 + -4.02945613e+00 1.14303496e+00 + -1.92694576e-01 1.03301431e-01 + 1.89554730e+00 -4.60128096e-01 + -2.55626581e+00 1.16057084e+00 + 6.89144365e-01 -9.94982900e-01 + -4.44680606e+00 2.19751983e+00 + -3.15196193e+00 1.18762993e+00 + -1.17434977e+00 1.04534656e+00 + 8.58386984e-02 -1.03947487e+00 + 3.33354973e-01 5.54813610e-01 + -9.37631808e-01 3.33450150e-01 + -2.50232471e+00 5.39720635e-01 + 1.03611949e+00 -7.16304095e-02 + -2.05556816e-02 -3.28992265e-01 + -2.24176201e+00 1.13077506e+00 + 4.53583688e+00 -1.10710212e+00 + 4.77389762e-01 -8.99445512e-01 + -2.69075551e+00 6.83176866e-01 + -2.21779724e+00 1.16916849e+00 + -1.09669056e+00 2.10044765e-01 + -8.45367920e-01 -8.45951423e-02 + 4.37558941e-01 -6.95904256e-01 + 1.84884195e+00 -1.71205136e-01 + -8.36371957e-01 5.62862478e-01 + 1.27786531e+00 -1.33362147e+00 + 2.90684492e+00 -7.49892184e-01 + -3.38652716e+00 1.51180670e+00 + -1.30945978e+00 7.09261928e-01 + -7.50471924e-01 -5.24637889e-01 + 1.18580718e+00 -9.97943971e-04 + -7.55395645e+00 3.19273590e+00 + 1.72822535e+00 -1.20996962e+00 + 5.67374320e-01 6.19573416e-01 + -2.99163781e+00 1.79721534e+00 + 1.49862187e+00 -6.05631846e-02 + 1.79503506e+00 -4.90419706e-01 + 3.85626054e+00 -1.95396324e+00 + -9.39188410e-01 7.96498057e-01 + 2.91986664e+00 -1.29392724e+00 + -1.54265750e+00 6.40727933e-01 + 1.14919794e+00 1.20834257e-01 + 2.00936817e+00 -1.53728359e+00 + 3.72468420e+00 -1.38704612e+00 + -1.27794802e+00 3.48543179e-01 + 3.63294077e-01 5.70623314e-01 + 1.49381016e+00 -6.04500534e-01 + 2.98912256e+00 -1.72295726e+00 + -1.80833817e+00 2.94907625e-01 + -3.19669622e+00 1.31888700e+00 + 1.45889401e+00 -8.88448639e-01 + -2.80045388e+00 1.01207060e+00 + -4.78379567e+00 1.48646520e+00 + 2.25510003e+00 -7.13372461e-01 + -9.74441433e-02 -2.17766373e-01 + 2.64468496e-01 -3.60842698e-01 + -5.98821713e+00 3.20197892e+00 + 2.67030213e-01 -5.36386416e-01 + 2.24546960e+00 -8.13464649e-01 + -4.89171414e-01 3.86255031e-01 + -7.45713706e-01 6.29800380e-01 + -3.30460503e-01 3.85127284e-01 + -4.19588147e+00 1.52793198e+00 + 5.42078582e-01 -2.61642741e-02 + 4.24938513e-01 -5.72936751e-01 + 2.82717288e+00 -6.75355024e-01 + -1.44741788e+00 5.03578028e-01 + -1.65547573e+00 7.76444277e-01 + 2.20361170e+00 -1.40835680e+00 + -3.69540235e+00 2.32953767e+00 + -1.41909357e-01 2.28989778e-01 + 1.92838879e+00 -8.72525737e-01 + 1.40708100e+00 -6.81849638e-02 + 1.24988112e+00 -1.39470590e-01 + -2.39435855e+00 7.26587655e-01 + 7.03985028e-01 4.85403277e-02 + 4.05214529e+00 -9.16928318e-01 + 3.74198837e-01 -5.04192358e-01 + -8.43374127e-01 2.36064018e-01 + -3.32253349e-01 7.47840055e-01 + -6.03725210e+00 1.95173337e+00 + 4.60829865e+00 -1.51191309e+00 + -1.46247098e+00 1.11140916e+00 + -9.60111157e-01 -1.23189114e-01 + -7.49613187e-01 4.53614129e-01 + -5.77838219e-01 2.07366469e-02 + 8.07652950e-01 -5.16272662e-01 + -6.02556049e-01 5.05318649e-01 + -1.28712445e-01 2.57836512e-01 + -5.27662820e+00 2.11790737e+00 + 5.40819308e+00 -2.15366022e+00 + 9.37742513e-02 -1.60221751e-01 + 4.55902865e+00 -1.24646307e+00 + -9.06582589e-01 1.92928110e-01 + 2.99928996e+00 -8.04301218e-01 + -3.24317381e+00 1.80076061e+00 + 3.20421743e-01 8.76524679e-01 + -5.29606705e-01 -3.16717696e-01 + -1.77264560e+00 7.52686776e-01 + -1.51706824e+00 8.43755103e-01 + 1.52759111e+00 -7.86814243e-01 + 4.74845617e-01 4.21319700e-01 + 6.97829149e-01 -8.15664881e-01 + 3.09564973e+00 -1.06202469e+00 + 2.95320379e+00 -1.98963943e+00 + -4.23033224e+00 1.41013338e+00 + 1.48576206e+00 8.02908511e-02 + 4.52041627e+00 -2.04620399e+00 + 6.58403922e-01 -7.60781799e-01 + 2.10667543e-01 1.15241731e-01 + 1.77702583e+00 -8.10271859e-01 + 2.41277385e+00 -1.46972042e+00 + 1.50685525e+00 -1.99272545e-01 + 7.61665522e-01 -4.11276152e-01 + 1.18352312e+00 -9.59908608e-01 + -3.32031305e-01 8.07500132e-02 + 1.16813118e+00 -1.73095194e-01 + 1.18363346e+00 -5.41565052e-01 + 5.17702179e-01 -7.62442035e-01 + 4.57401006e-01 -1.45951115e-02 + 1.49377115e-01 2.99571605e-01 + 1.40399453e+00 -1.30160353e+00 + 5.26231567e-01 3.52783752e-01 + -1.91136514e+00 4.24228635e-01 + 1.74156701e+00 -9.92076776e-01 + -4.89323391e+00 2.32483507e+00 + 2.54011209e+00 -8.80366295e-01 + -5.56925706e-01 1.48842026e-01 + -2.35904668e+00 9.60474853e-01 + 1.42216971e+00 -4.67062761e-01 + -1.10809680e+00 7.68684300e-01 + 4.09674726e+00 -1.90795680e+00 + -2.23048923e+00 9.03812542e-01 + 6.57025763e-01 1.36514871e-01 + 2.10944145e+00 -9.78897838e-02 + 1.22552525e+00 -2.50303867e-01 + 2.84620103e-01 -5.30164020e-01 + -2.13562585e+00 1.03503056e+00 + 1.32414902e-01 -8.14190240e-03 + -5.82433561e-01 3.21020292e-01 + -5.06473247e-01 3.11530419e-01 + 1.57162465e+00 -1.20763919e+00 + -1.43155284e+00 -2.51203698e-02 + -1.47093713e+00 -1.39620999e-01 + -2.65765643e+00 1.06091403e+00 + 2.45992927e+00 -5.88815836e-01 + -1.28440162e+00 -1.99377398e-01 + 6.11257504e-01 -3.73577401e-01 + -3.46606103e-01 6.06081290e-01 + 3.76687505e+00 -8.80181424e-01 + -1.03725103e+00 1.45177517e+00 + 2.76659936e+00 -1.09361320e+00 + -3.61311296e+00 9.75032455e-01 + 3.22878655e+00 -9.69497365e-01 + 1.43560379e+00 -5.52524585e-01 + 2.94042153e+00 -1.79747037e+00 + 1.30739580e+00 2.47989248e-01 + -4.05056982e-01 1.22831715e+00 + -2.25827421e+00 2.30604626e-01 + 3.69262926e-01 4.32714650e-02 + -5.52064063e-01 6.07806340e-01 + 7.03325987e+00 -2.17956730e+00 + -2.37823835e-01 -8.28068639e-01 + -4.84279888e-01 5.67765194e-01 + -3.15863410e+00 1.02241617e+00 + -3.39561593e+00 1.36876374e+00 + -2.78482934e+00 6.81641104e-01 + -4.37604334e+00 2.23826340e+00 + -2.54049692e+00 8.22676745e-01 + 3.73264822e+00 -9.93498732e-01 + -3.49536064e+00 1.84771519e+00 + 9.81801604e-01 -5.21278776e-01 + 1.52996831e+00 -1.27386206e+00 + -9.23490293e-01 5.29099482e-01 + -2.76999461e+00 9.24831872e-01 + -3.30029834e-01 -2.49645555e-01 + -1.71156166e+00 5.44940854e-01 + -2.37009487e+00 5.83826982e-01 + -3.03216865e+00 1.04922722e+00 + -2.19539936e+00 1.37558730e+00 + 1.15350207e+00 -6.15318535e-01 + 4.62011792e+00 -2.46714517e+00 + 1.52627952e-02 -1.00618283e-01 + -1.10399342e+00 4.87413533e-01 + 3.55448194e+00 -9.10394190e-01 + -5.21890321e+00 2.44710745e+00 + 1.54289749e+00 -6.54269311e-01 + 2.67935674e+00 -9.92758863e-01 + 1.05801310e+00 2.60054285e-02 + 1.52509097e+00 -4.08768600e-01 + 3.27576917e+00 -1.28769406e+00 + 1.71008412e-01 -2.68739994e-01 + -9.83351344e-04 7.02495897e-02 + -7.60795056e-03 1.61968285e-01 + -1.80620472e+00 4.24934471e-01 + 2.32023297e-02 -2.57284559e-01 + 3.98219478e-01 -4.65361935e-01 + 6.63476988e-01 -3.29823196e-02 + 4.00154707e+00 -1.01792211e+00 + -1.50286870e+00 9.46875359e-01 + -2.22717585e+00 7.50636195e-01 + -3.47381508e-01 -6.51596975e-01 + 2.08076453e+00 -8.22800165e-01 + 2.05099963e+00 -4.00868250e-01 + 3.52576988e-02 -2.54418565e-01 + 1.57342042e+00 -7.62166492e-02 + -1.47019722e+00 3.40861172e-01 + -1.21156090e+00 3.21891246e-01 + 3.79729047e+00 -1.54350764e+00 + 1.26459678e-02 6.99203693e-01 + 1.53974177e-01 4.68643204e-01 + -1.73923561e-01 -1.26229768e-01 + 4.54644993e+00 -2.13951783e+00 + 1.46022547e-01 -4.57084165e-01 + 6.50048037e+00 -2.78872609e+00 + -1.51934912e+00 1.03216768e+00 + -3.06483575e+00 1.81101446e+00 + -2.38212125e+00 9.19559042e-01 + -1.81319611e+00 8.10545112e-01 + 1.70951294e+00 -6.10712680e-01 + 1.67974156e+00 -1.51241453e+00 + -5.94795113e+00 2.56893813e+00 + 3.62633110e-01 -7.46965304e-01 + -2.44042594e+00 8.52761797e-01 + 3.32412550e+00 -1.28439899e+00 + 4.74860766e+00 -1.72821964e+00 + 1.29072541e+00 -8.24872902e-01 + -1.69450702e+00 4.09600876e-01 + 1.29705411e+00 1.22300809e-01 + -2.63597613e+00 8.55612913e-01 + 9.28467301e-01 -2.63550114e-02 + 2.44670264e+00 -4.10123002e-01 + 1.06408206e+00 -5.03361942e-01 + 5.12384049e-02 -1.27116595e-02 + -1.06731272e+00 -1.76205029e-01 + -9.45454582e-01 3.74404917e-01 + 2.54343689e+00 -7.13810545e-01 + -2.54460335e+00 1.31590265e+00 + 1.89864233e+00 -3.98436339e-01 + -1.93990133e+00 6.01474630e-01 + -1.35938824e+00 4.00751788e-01 + 2.38567018e+00 -6.13904880e-01 + 2.18748050e-01 2.62631712e-01 + -2.01388788e+00 1.41474031e+00 + 2.74014581e+00 -1.27448105e+00 + -2.13828583e+00 1.13616144e+00 + 5.98730932e+00 -2.53430080e+00 + -1.72872795e+00 1.53702057e+00 + -2.53263962e+00 1.27342410e+00 + 1.34326968e+00 -1.99395088e-01 + 3.83352666e-01 -1.25683065e-01 + -2.35630657e+00 5.54116983e-01 + -1.94900838e+00 5.76270178e-01 + -1.36699108e+00 -3.40904824e-01 + -2.34727346e+00 -1.93054940e-02 + -3.82779777e+00 1.83025664e+00 + -4.31602080e+00 9.21605705e-01 + 5.54098133e-01 2.33991419e-01 + -4.53591188e+00 1.99833353e+00 + -3.92715909e+00 1.83231482e+00 + 3.91344440e-01 -1.11355111e-01 + 3.48576363e+00 -1.41379449e+00 + -1.42858690e+00 3.84532286e-01 + 1.79519859e+00 -9.23486448e-01 + 8.49691242e-01 -1.76551084e-01 + 1.53618138e+00 8.23835015e-02 + 5.91476520e-02 3.88296940e-02 + 1.44837346e+00 -7.24097604e-01 + -6.79008418e-01 4.04078097e-01 + 2.87555510e+00 -9.51825076e-01 + -1.12379101e+00 2.93457714e-01 + 1.45263980e+00 -6.01960544e-01 + -2.55741621e-01 9.26233518e-01 + 3.54570714e+00 -1.41521877e+00 + -1.61542388e+00 6.57844512e-01 + -3.22844269e-01 3.02823546e-01 + 1.03523913e+00 -6.92730711e-01 + 1.11084909e+00 -3.50823642e-01 + 3.41268693e+00 -1.90865862e+00 + 7.67062858e-01 -9.48792160e-01 + -5.49798016e+00 1.71139960e+00 + 1.14865798e+00 -6.12669150e-01 + -2.18256680e+00 7.78634462e-01 + 4.78857389e+00 -2.55555085e+00 + -1.85555569e+00 8.04311615e-01 + -4.22278799e+00 2.01162524e+00 + -1.56556149e+00 1.54353907e+00 + -3.11527864e+00 1.65973526e+00 + 2.66342611e+00 -1.20449402e+00 + 1.57635314e+00 -1.48716308e-01 + -6.35606865e-01 2.59701180e-01 + 1.02431976e+00 -6.76929904e-01 + 1.12973772e+00 1.49473892e-02 + -9.12758116e-01 2.21533933e-01 + -2.98014470e+00 1.71651189e+00 + 2.74016965e+00 -9.47893923e-01 + -3.47830591e+00 1.34941430e+00 + 1.74757562e+00 -3.72503752e-01 + 5.55820383e-01 -6.47992466e-01 + -1.19871928e+00 9.82429151e-01 + -2.53040133e+00 2.10671307e+00 + -1.94085605e+00 1.38938137e+00 diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 413b824e369da..1e9ef345b7435 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -16,9 +16,12 @@ often used for exploratory analysis and/or as a component of a hierarchical supervised learning pipeline (in which distinct classifiers or regression models are trained for each cluster). -MLlib supports -[k-means](http://en.wikipedia.org/wiki/K-means_clustering) clustering, one of -the most commonly used clustering algorithms that clusters the data points into +MLlib supports the following models: + +### 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 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). @@ -34,6 +37,20 @@ 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: @@ -56,6 +73,8 @@ Example outputs for a dataset inspired by the paper - but with five clusters ins ### Examples +#### k-means +

The following code snippets can be executed in `spark-shell`. @@ -168,6 +187,112 @@ print("Within Set Sum of Squared Error = " + str(WSSSE))
+#### GaussianMixture + +
+
+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 +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.linalg.Vectors + +// Load and parse the data +val data = sc.textFile("data/mllib/gmm_data.txt") +val parsedData = data.map(s => Vectors.dense(s.trim.split(' ').map(_.toDouble))).cache() + +// Cluster the data into two classes using GaussianMixture +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 + (gmm.weights(i), gmm.gaussians(i).mu, gmm.gaussians(i).sigma)) +} + +{% endhighlight %} +
+ +
+All of MLlib's methods use Java-friendly types, so you can import and call them there the same +way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the +Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by +calling `.rdd()` on your `JavaRDD` object. A self-contained application example +that is equivalent to the provided example in Scala is given below: + +{% highlight java %} +import org.apache.spark.api.java.*; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.clustering.GaussianMixture; +import org.apache.spark.mllib.clustering.GaussianMixtureModel; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.SparkConf; + +public class GaussianMixtureExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("GaussianMixture Example"); + JavaSparkContext sc = new JavaSparkContext(conf); + + // Load and parse data + String path = "data/mllib/gmm_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); + } + } + ); + parsedData.cache(); + + // Cluster the data into two classes using GaussianMixture + GaussianMixtureModel gmm = new GaussianMixture().setK(2).run(parsedData.rdd()); + + // Output the parameters of the mixture model + for(int j=0; j + +
+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 +to the algorithm. We then output the parameters of the mixture model. + +{% highlight python %} +from pyspark.mllib.clustering import GaussianMixture +from numpy import array + +# Load and parse the data +data = sc.textFile("data/mllib/gmm_data.txt") +parsedData = data.map(lambda line: array([float(x) for x in line.strip().split(' ')])) + +# Build the model (cluster the data) +gmm = GaussianMixture.train(parsedData, 2) + +# output parameters of model +for i in range(2): + print ("weight = ", gmm.weights[i], "mu = ", gmm.gaussians[i].mu, + "sigma = ", gmm.gaussians[i].sigma.toArray()) + +{% 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/docs/mllib-guide.md b/docs/mllib-guide.md index 73728bb35eb96..7779fbc9c49e4 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -25,6 +25,8 @@ filtering, dimensionality reduction, as well as underlying optimization primitiv * alternating least squares (ALS) * [Clustering](mllib-clustering.html) * k-means + * Gaussian mixture + * power iteration * [Dimensionality reduction](mllib-dimensionality-reduction.html) * singular value decomposition (SVD) * principal component analysis (PCA) From cc6e53119d7a51b95b19244f50b25814088b4d11 Mon Sep 17 00:00:00 2001 From: lianhuiwang Date: Fri, 6 Feb 2015 10:48:31 -0800 Subject: [PATCH 117/578] [SPARK-5653][YARN] In ApplicationMaster rename isDriver to isClusterMode in ApplicationMaster rename isDriver to isClusterMode,because in Client it uses isClusterMode,ApplicationMaster should keep consistent with it and uses isClusterMode.Also isClusterMode is easier to understand. andrewor14 sryza Author: lianhuiwang Closes #4430 from lianhuiwang/am-isDriver-rename and squashes the following commits: f9f3ed0 [lianhuiwang] rename isDriver to isClusterMode --- .../spark/deploy/yarn/ApplicationMaster.scala | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 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 37e98e01fddf7..4cc320c5d59b5 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 @@ -54,7 +54,7 @@ private[spark] class ApplicationMaster( private val sparkConf = new SparkConf() private val yarnConf: YarnConfiguration = SparkHadoopUtil.get.newConfiguration(sparkConf) .asInstanceOf[YarnConfiguration] - private val isDriver = args.userClass != null + private val isClusterMode = args.userClass != null // Default to numExecutors * 2, with minimum of 3 private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", @@ -81,7 +81,7 @@ private[spark] class ApplicationMaster( try { val appAttemptId = client.getAttemptId() - if (isDriver) { + if (isClusterMode) { // Set the web ui port to be ephemeral for yarn so we don't conflict with // other spark processes running on the same box System.setProperty("spark.ui.port", "0") @@ -139,7 +139,7 @@ private[spark] class ApplicationMaster( // doAs in order for the credentials to be passed on to the executor containers. val securityMgr = new SecurityManager(sparkConf) - if (isDriver) { + if (isClusterMode) { runDriver(securityMgr) } else { runExecutorLauncher(securityMgr) @@ -162,7 +162,7 @@ private[spark] class ApplicationMaster( * from the application code. */ final def getDefaultFinalStatus() = { - if (isDriver) { + if (isClusterMode) { FinalApplicationStatus.SUCCEEDED } else { FinalApplicationStatus.UNDEFINED @@ -243,7 +243,7 @@ private[spark] class ApplicationMaster( private def runAMActor( host: String, port: String, - isDriver: Boolean): Unit = { + isClusterMode: Boolean): Unit = { val driverUrl = AkkaUtils.address( AkkaUtils.protocol(actorSystem), @@ -251,7 +251,7 @@ private[spark] class ApplicationMaster( host, port, YarnSchedulerBackend.ACTOR_NAME) - actor = actorSystem.actorOf(Props(new AMActor(driverUrl, isDriver)), name = "YarnAM") + actor = actorSystem.actorOf(Props(new AMActor(driverUrl, isClusterMode)), name = "YarnAM") } private def runDriver(securityMgr: SecurityManager): Unit = { @@ -272,7 +272,7 @@ private[spark] class ApplicationMaster( runAMActor( sc.getConf.get("spark.driver.host"), sc.getConf.get("spark.driver.port"), - isDriver = true) + isClusterMode = true) registerAM(sc.ui.map(_.appUIAddress).getOrElse(""), securityMgr) userClassThread.join() } @@ -427,7 +427,7 @@ private[spark] class ApplicationMaster( sparkConf.set("spark.driver.host", driverHost) sparkConf.set("spark.driver.port", driverPort.toString) - runAMActor(driverHost, driverPort.toString, isDriver = false) + runAMActor(driverHost, driverPort.toString, isClusterMode = false) } /** Add the Yarn IP filter that is required for properly securing the UI. */ @@ -435,7 +435,7 @@ private[spark] class ApplicationMaster( val proxyBase = System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" val params = client.getAmIpFilterParams(yarnConf, proxyBase) - if (isDriver) { + if (isClusterMode) { System.setProperty("spark.ui.filters", amFilter) params.foreach { case (k, v) => System.setProperty(s"spark.$amFilter.param.$k", v) } } else { @@ -491,7 +491,7 @@ private[spark] class ApplicationMaster( /** * An actor that communicates with the driver's scheduler backend. */ - private class AMActor(driverUrl: String, isDriver: Boolean) extends Actor { + private class AMActor(driverUrl: String, isClusterMode: Boolean) extends Actor { var driver: ActorSelection = _ override def preStart() = { @@ -503,7 +503,7 @@ private[spark] class ApplicationMaster( 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 (!isDriver) { + if (!isClusterMode) { context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) } } @@ -513,7 +513,7 @@ private[spark] class ApplicationMaster( 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 (!isDriver) { + if (!isClusterMode) { finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) } From 1a88f20de798030a7d5713bd267f612ba5617fca Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 6 Feb 2015 10:53:16 -0800 Subject: [PATCH 118/578] SPARK-4337. [YARN] Add ability to cancel pending requests Author: Sandy Ryza Closes #4141 from sryza/sandy-spark-4337 and squashes the following commits: a98bd20 [Sandy Ryza] Andrew's comments cdaab7f [Sandy Ryza] SPARK-4337. Add ability to cancel pending requests to YARN --- .../spark/deploy/yarn/YarnAllocator.scala | 65 ++++++++++++------- .../deploy/yarn/YarnAllocatorSuite.scala | 54 +++++++++++++-- 2 files changed, 89 insertions(+), 30 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 0dbb6154b3039..12c62a659d799 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 @@ -69,8 +69,7 @@ private[yarn] class YarnAllocator( } // Visible for testing. - val allocatedHostToContainersMap = - new HashMap[String, collection.mutable.Set[ContainerId]] + val allocatedHostToContainersMap = new HashMap[String, collection.mutable.Set[ContainerId]] val allocatedContainerToHostMap = new HashMap[ContainerId, String] // Containers that we no longer care about. We've either already told the RM to release them or @@ -84,7 +83,7 @@ private[yarn] class YarnAllocator( private var executorIdCounter = 0 @volatile private var numExecutorsFailed = 0 - @volatile private var maxExecutors = args.numExecutors + @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] @@ -133,10 +132,12 @@ private[yarn] class YarnAllocator( amClient.getMatchingRequests(RM_REQUEST_PRIORITY, location, resource).map(_.size).sum /** - * Request as many executors from the ResourceManager as needed to reach the desired total. + * Request as many executors from the ResourceManager as needed to reach the desired total. If + * the requested total is smaller than the current number of running executors, no executors will + * be killed. */ def requestTotalExecutors(requestedTotal: Int): Unit = synchronized { - maxExecutors = requestedTotal + targetNumExecutors = requestedTotal } /** @@ -147,8 +148,8 @@ private[yarn] class YarnAllocator( val container = executorIdToContainer.remove(executorId).get internalReleaseContainer(container) numExecutorsRunning -= 1 - maxExecutors -= 1 - assert(maxExecutors >= 0, "Allocator killed more executors than are allocated!") + targetNumExecutors -= 1 + assert(targetNumExecutors >= 0, "Allocator killed more executors than are allocated!") } else { logWarning(s"Attempted to kill unknown executor $executorId!") } @@ -163,15 +164,8 @@ private[yarn] class YarnAllocator( * This must be synchronized because variables read in this method are mutated by other methods. */ def allocateResources(): Unit = synchronized { - val numPendingAllocate = getNumPendingAllocate - val missing = maxExecutors - numPendingAllocate - numExecutorsRunning + updateResourceRequests() - if (missing > 0) { - logInfo(s"Will request $missing executor containers, each with ${resource.getVirtualCores} " + - s"cores and ${resource.getMemory} MB memory including $memoryOverhead MB overhead") - } - - addResourceRequests(missing) val progressIndicator = 0.1f // Poll the ResourceManager. This doubles as a heartbeat if there are no pending container // requests. @@ -201,15 +195,36 @@ private[yarn] class YarnAllocator( } /** - * Request numExecutors additional containers from YARN. Visible for testing. + * Update the set of container requests that we will sync with the RM based on the number of + * executors we have currently running and our target number of executors. + * + * Visible for testing. */ - def addResourceRequests(numExecutors: Int): Unit = { - for (i <- 0 until numExecutors) { - val request = new ContainerRequest(resource, null, null, RM_REQUEST_PRIORITY) - amClient.addContainerRequest(request) - val nodes = request.getNodes - val hostStr = if (nodes == null || nodes.isEmpty) "Any" else nodes.last - logInfo("Container request (host: %s, capability: %s".format(hostStr, resource)) + def updateResourceRequests(): Unit = { + val numPendingAllocate = getNumPendingAllocate + val missing = targetNumExecutors - numPendingAllocate - numExecutorsRunning + + if (missing > 0) { + logInfo(s"Will request $missing executor containers, each with ${resource.getVirtualCores} " + + s"cores and ${resource.getMemory} MB memory including $memoryOverhead MB overhead") + + for (i <- 0 until missing) { + val request = new ContainerRequest(resource, null, null, RM_REQUEST_PRIORITY) + amClient.addContainerRequest(request) + val nodes = request.getNodes + val hostStr = if (nodes == null || nodes.isEmpty) "Any" else nodes.last + logInfo(s"Container request (host: $hostStr, capability: $resource)") + } + } else if (missing < 0) { + val numToCancel = math.min(numPendingAllocate, -missing) + logInfo(s"Canceling requests for $numToCancel executor containers") + + val matchingRequests = amClient.getMatchingRequests(RM_REQUEST_PRIORITY, ANY_HOST, resource) + if (!matchingRequests.isEmpty) { + matchingRequests.head.take(numToCancel).foreach(amClient.removeContainerRequest) + } else { + logWarning("Expected to find pending requests, but found none.") + } } } @@ -266,7 +281,7 @@ private[yarn] class YarnAllocator( * containersToUse or remaining. * * @param allocatedContainer container that was given to us by YARN - * @location resource name, either a node, rack, or * + * @param location resource name, either a node, rack, or * * @param containersToUse list of containers that will be used * @param remaining list of containers that will not be used */ @@ -294,7 +309,7 @@ private[yarn] class YarnAllocator( private def runAllocatedContainers(containersToUse: ArrayBuffer[Container]): Unit = { for (container <- containersToUse) { numExecutorsRunning += 1 - assert(numExecutorsRunning <= maxExecutors) + assert(numExecutorsRunning <= targetNumExecutors) val executorHostname = container.getNodeId.getHost val containerId = container.getId executorIdCounter += 1 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 024b25f9d3365..3c224f148802e 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 @@ -107,8 +107,8 @@ class YarnAllocatorSuite extends FunSuite with Matchers with BeforeAndAfterEach test("single container allocated") { // request a single container and receive it - val handler = createAllocator() - handler.addResourceRequests(1) + val handler = createAllocator(1) + handler.updateResourceRequests() handler.getNumExecutorsRunning should be (0) handler.getNumPendingAllocate should be (1) @@ -123,8 +123,8 @@ class YarnAllocatorSuite extends FunSuite with Matchers with BeforeAndAfterEach test("some containers allocated") { // request a few containers and receive some of them - val handler = createAllocator() - handler.addResourceRequests(4) + val handler = createAllocator(4) + handler.updateResourceRequests() handler.getNumExecutorsRunning should be (0) handler.getNumPendingAllocate should be (4) @@ -144,7 +144,7 @@ class YarnAllocatorSuite extends FunSuite with Matchers with BeforeAndAfterEach test("receive more containers than requested") { val handler = createAllocator(2) - handler.addResourceRequests(2) + handler.updateResourceRequests() handler.getNumExecutorsRunning should be (0) handler.getNumPendingAllocate should be (2) @@ -162,6 +162,50 @@ class YarnAllocatorSuite extends FunSuite with Matchers with BeforeAndAfterEach handler.allocatedHostToContainersMap.contains("host4") should be (false) } + test("decrease total requested executors") { + val handler = createAllocator(4) + handler.updateResourceRequests() + handler.getNumExecutorsRunning should be (0) + handler.getNumPendingAllocate should be (4) + + handler.requestTotalExecutors(3) + handler.updateResourceRequests() + handler.getNumPendingAllocate should be (3) + + val container = createContainer("host1") + handler.handleAllocatedContainers(Array(container)) + + handler.getNumExecutorsRunning should be (1) + handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1") + handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId) + + handler.requestTotalExecutors(2) + handler.updateResourceRequests() + handler.getNumPendingAllocate should be (1) + } + + test("decrease total requested executors to less than currently running") { + val handler = createAllocator(4) + handler.updateResourceRequests() + handler.getNumExecutorsRunning should be (0) + handler.getNumPendingAllocate should be (4) + + handler.requestTotalExecutors(3) + handler.updateResourceRequests() + handler.getNumPendingAllocate should be (3) + + val container1 = createContainer("host1") + val container2 = createContainer("host2") + handler.handleAllocatedContainers(Array(container1, container2)) + + handler.getNumExecutorsRunning should be (2) + + handler.requestTotalExecutors(1) + handler.updateResourceRequests() + handler.getNumPendingAllocate should be (0) + handler.getNumExecutorsRunning should be (2) + } + test("memory exceeded diagnostic regexes") { val diagnostics = "Container [pid=12465,containerID=container_1412887393566_0003_01_000002] is running " + From fe3740c4c859d087b714c666741a29061bba5f58 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 6 Feb 2015 10:54:23 -0800 Subject: [PATCH 119/578] [SPARK-5636] Ramp up faster in dynamic allocation A recent patch #4051 made the initial number default to 0. With this change, any Spark application using dynamic allocation's default settings will ramp up very slowly. Since we never request more executors than needed to saturate the pending tasks, it is safe to ramp up quickly. The current default of 60 may be too slow. Author: Andrew Or Closes #4409 from andrewor14/dynamic-allocation-interval and squashes the following commits: d3cc485 [Andrew Or] Lower request interval --- .../scala/org/apache/spark/ExecutorAllocationManager.scala | 6 +++--- docs/configuration.md | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 5d5288bb6e60d..8b38366e0325c 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -76,15 +76,15 @@ private[spark] class ExecutorAllocationManager( private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", Integer.MAX_VALUE) - // How long there must be backlogged tasks for before an addition is triggered + // How long there must be backlogged tasks for before an addition is triggered (seconds) private val schedulerBacklogTimeout = conf.getLong( - "spark.dynamicAllocation.schedulerBacklogTimeout", 60) + "spark.dynamicAllocation.schedulerBacklogTimeout", 5) // Same as above, but used only after `schedulerBacklogTimeout` is exceeded private val sustainedSchedulerBacklogTimeout = conf.getLong( "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout) - // How long an executor must be idle for before it is removed + // How long an executor must be idle for before it is removed (seconds) private val executorIdleTimeout = conf.getLong( "spark.dynamicAllocation.executorIdleTimeout", 600) diff --git a/docs/configuration.md b/docs/configuration.md index 4c86cb7c16238..00e973c245005 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1140,7 +1140,7 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.schedulerBacklogTimeout - 60 + 5 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 From c01b9852ea2f7d453249b07d89e62af71bd26e3d Mon Sep 17 00:00:00 2001 From: Masayoshi TSUZUKI Date: Fri, 6 Feb 2015 10:58:26 -0800 Subject: [PATCH 120/578] [SPARK-5396] Syntax error in spark scripts on windows. Modified syntax error in spark-submit2.cmd. Command prompt doesn't have "defined" operator. Author: Masayoshi TSUZUKI Closes #4428 from tsudukim/feature/SPARK-5396 and squashes the following commits: ec18465 [Masayoshi TSUZUKI] [SPARK-5396] Syntax error in spark scripts on windows. --- bin/spark-submit2.cmd | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/spark-submit2.cmd b/bin/spark-submit2.cmd index 12244a9cb04fb..446cbc74b74f9 100644 --- a/bin/spark-submit2.cmd +++ b/bin/spark-submit2.cmd @@ -25,7 +25,7 @@ set ORIG_ARGS=%* rem Reset the values of all variables used set SPARK_SUBMIT_DEPLOY_MODE=client -if not defined %SPARK_CONF_DIR% ( +if [%SPARK_CONF_DIR%] == [] ( set SPARK_CONF_DIR=%SPARK_HOME%\conf ) set SPARK_SUBMIT_PROPERTIES_FILE=%SPARK_CONF_DIR%\spark-defaults.conf From 9792bec596113a6f5f4534772b7539255403b082 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Fri, 6 Feb 2015 11:03:56 -0800 Subject: [PATCH 121/578] [SPARK-4877] Allow user first classes to extend classes in the parent. Previously, the classloader isolation was almost too good, such that if a child class needed to load/reference a class that was only available in the parent, it could not do so. This adds tests for that case, the user-first Fake2 class extends the only-in-parent Fake3 class. It also sneaks in a fix where only the first stage seemed to work, and on subsequent stages, a LinkageError happened because classes from the user-first classpath were getting defined twice. Author: Stephen Haberman Closes #3725 from stephenh/4877_user_first_parent_inheritance and squashes the following commits: dabcd35 [Stephen Haberman] [SPARK-4877] Respect userClassPathFirst for the driver code too. 3d0fa7c [Stephen Haberman] [SPARK-4877] Allow user first classes to extend classes in the parent. --- .../scala/org/apache/spark/TestUtils.scala | 34 +++++++++++++++---- .../org/apache/spark/deploy/SparkSubmit.scala | 17 ++++++---- .../executor/ExecutorURLClassLoader.scala | 12 ++++++- .../ExecutorURLClassLoaderSuite.scala | 16 ++++++--- 4 files changed, 61 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 34078142f5385..be081c3825566 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -43,11 +43,20 @@ private[spark] object TestUtils { * Note: if this is used during class loader tests, class names should be unique * in order to avoid interference between tests. */ - def createJarWithClasses(classNames: Seq[String], value: String = ""): URL = { + def createJarWithClasses( + classNames: Seq[String], + toStringValue: String = "", + classNamesWithBase: Seq[(String, String)] = Seq(), + classpathUrls: Seq[URL] = Seq()): URL = { val tempDir = Utils.createTempDir() - val files = for (name <- classNames) yield createCompiledClass(name, tempDir, value) + val files1 = for (name <- classNames) yield { + createCompiledClass(name, tempDir, toStringValue, classpathUrls = classpathUrls) + } + val files2 = for ((childName, baseName) <- classNamesWithBase) yield { + createCompiledClass(childName, tempDir, toStringValue, baseName, classpathUrls) + } val jarFile = new File(tempDir, "testJar-%s.jar".format(System.currentTimeMillis())) - createJar(files, jarFile) + createJar(files1 ++ files2, jarFile) } @@ -85,15 +94,26 @@ private[spark] object TestUtils { } /** Creates a compiled class with the given name. Class file will be placed in destDir. */ - def createCompiledClass(className: String, destDir: File, value: String = ""): File = { + def createCompiledClass( + className: String, + destDir: File, + toStringValue: String = "", + baseClass: String = null, + classpathUrls: Seq[URL] = Seq()): File = { val compiler = ToolProvider.getSystemJavaCompiler + val extendsText = Option(baseClass).map { c => s" extends ${c}" }.getOrElse("") val sourceFile = new JavaSourceFromString(className, - "public class " + className + " implements java.io.Serializable {" + - " @Override public String toString() { return \"" + value + "\"; }}") + "public class " + className + extendsText + " implements java.io.Serializable {" + + " @Override public String toString() { return \"" + toStringValue + "\"; }}") // Calling this outputs a class file in pwd. It's easier to just rename the file than // build a custom FileManager that controls the output location. - compiler.getTask(null, null, null, null, null, Seq(sourceFile)).call() + val options = if (classpathUrls.nonEmpty) { + Seq("-classpath", classpathUrls.map { _.getFile }.mkString(File.pathSeparator)) + } else { + Seq() + } + compiler.getTask(null, null, null, options, null, Seq(sourceFile)).call() val fileName = className + ".class" val result = new File(fileName) 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 8bbfcd2997dc6..9d25e647f1703 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -20,11 +20,9 @@ package org.apache.spark.deploy import java.io.{File, PrintStream} import java.lang.reflect.{Modifier, InvocationTargetException} import java.net.URL - import scala.collection.mutable.{ArrayBuffer, HashMap, Map} import org.apache.hadoop.fs.Path - import org.apache.ivy.Ivy import org.apache.ivy.core.LogOptions import org.apache.ivy.core.module.descriptor.{DefaultExcludeRule, DefaultDependencyDescriptor, DefaultModuleDescriptor} @@ -35,9 +33,10 @@ import org.apache.ivy.core.retrieve.RetrieveOptions 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.executor.ExecutorURLClassLoader import org.apache.spark.util.Utils +import org.apache.spark.executor.ChildExecutorURLClassLoader +import org.apache.spark.executor.MutableURLClassLoader /** * Main gateway of launching a Spark application. @@ -389,8 +388,14 @@ object SparkSubmit { printStream.println("\n") } - val loader = new ExecutorURLClassLoader(new Array[URL](0), - Thread.currentThread.getContextClassLoader) + val loader = + if (sysProps.getOrElse("spark.files.userClassPathFirst", "false").toBoolean) { + new ChildExecutorURLClassLoader(new Array[URL](0), + Thread.currentThread.getContextClassLoader) + } else { + new ExecutorURLClassLoader(new Array[URL](0), + Thread.currentThread.getContextClassLoader) + } Thread.currentThread.setContextClassLoader(loader) for (jar <- childClasspath) { @@ -434,7 +439,7 @@ object SparkSubmit { } } - private def addJarToClasspath(localJar: String, loader: ExecutorURLClassLoader) { + private def addJarToClasspath(localJar: String, loader: MutableURLClassLoader) { val uri = Utils.resolveURI(localJar) uri.getScheme match { case "file" | "local" => diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala index 218ed7b5d2d39..8011e75944aac 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala @@ -39,7 +39,17 @@ private[spark] class ChildExecutorURLClassLoader(urls: Array[URL], parent: Class super.addURL(url) } override def findClass(name: String): Class[_] = { - super.findClass(name) + val loaded = super.findLoadedClass(name) + if (loaded != null) { + return loaded + } + try { + super.findClass(name) + } catch { + case e: ClassNotFoundException => { + parentClassLoader.loadClass(name) + } + } } } diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala index e2050e95a1b88..b7912c09d1410 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala @@ -26,10 +26,14 @@ import org.apache.spark.util.Utils class ExecutorURLClassLoaderSuite extends FunSuite { - val childClassNames = List("FakeClass1", "FakeClass2") - val parentClassNames = List("FakeClass1", "FakeClass2", "FakeClass3") - val urls = List(TestUtils.createJarWithClasses(childClassNames, "1")).toArray - val urls2 = List(TestUtils.createJarWithClasses(parentClassNames, "2")).toArray + val urls2 = List(TestUtils.createJarWithClasses( + classNames = Seq("FakeClass1", "FakeClass2", "FakeClass3"), + toStringValue = "2")).toArray + val urls = List(TestUtils.createJarWithClasses( + classNames = Seq("FakeClass1"), + classNamesWithBase = Seq(("FakeClass2", "FakeClass3")), // FakeClass3 is in parent + toStringValue = "1", + classpathUrls = urls2)).toArray test("child first") { val parentLoader = new URLClassLoader(urls2, null) @@ -37,6 +41,8 @@ class ExecutorURLClassLoaderSuite extends FunSuite { val fakeClass = classLoader.loadClass("FakeClass2").newInstance() val fakeClassVersion = fakeClass.toString assert(fakeClassVersion === "1") + val fakeClass2 = classLoader.loadClass("FakeClass2").newInstance() + assert(fakeClass.getClass === fakeClass2.getClass) } test("parent first") { @@ -45,6 +51,8 @@ class ExecutorURLClassLoaderSuite extends FunSuite { val fakeClass = classLoader.loadClass("FakeClass1").newInstance() val fakeClassVersion = fakeClass.toString assert(fakeClassVersion === "2") + val fakeClass2 = classLoader.loadClass("FakeClass1").newInstance() + assert(fakeClass.getClass === fakeClass2.getClass) } test("child first can fall back") { From 6072fcc14ee1a4eba793e725fcb2cb2ffebd5b60 Mon Sep 17 00:00:00 2001 From: lianhuiwang Date: Fri, 6 Feb 2015 11:09:37 -0800 Subject: [PATCH 122/578] [SPARK-5593][Core]Replace BlockManagerListener with ExecutorListener in ExecutorAllocationListener More strictly, in ExecutorAllocationListener, we need to replace onBlockManagerAdded, onBlockManagerRemoved with onExecutorAdded,onExecutorRemoved. because at some time, onExecutorAdded and onExecutorRemoved are more accurate to express these meanings. example at SPARK-5529, BlockManager has been removed,but executor is existed. andrewor14 sryza Author: lianhuiwang Closes #4369 from lianhuiwang/SPARK-5593 and squashes the following commits: 333367c [lianhuiwang] Replace BlockManagerListener with ExecutorListener in ExecutorAllocationListener --- .../spark/ExecutorAllocationManager.scala | 9 +++--- .../ExecutorAllocationManagerSuite.scala | 32 +++++++++---------- 2 files changed, 19 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 8b38366e0325c..02d54bf3b53cc 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -486,8 +486,8 @@ private[spark] class ExecutorAllocationManager( } } - override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = { - val executorId = blockManagerAdded.blockManagerId.executorId + override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { + val executorId = executorAdded.executorId if (executorId != SparkContext.DRIVER_IDENTIFIER) { // This guards against the race condition in which the `SparkListenerTaskStart` // event is posted before the `SparkListenerBlockManagerAdded` event, which is @@ -498,9 +498,8 @@ private[spark] class ExecutorAllocationManager( } } - override def onBlockManagerRemoved( - blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit = { - allocationManager.onExecutorRemoved(blockManagerRemoved.blockManagerId.executorId) + override def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit = { + allocationManager.onExecutorRemoved(executorRemoved.executorId) } /** diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 57081ddd959a5..c2869628afbb3 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -22,7 +22,7 @@ import scala.collection.mutable import org.scalatest.{FunSuite, PrivateMethodTester} import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ -import org.apache.spark.storage.BlockManagerId +import org.apache.spark.scheduler.cluster.ExecutorInfo /** * Test add and remove behavior of ExecutorAllocationManager. @@ -144,8 +144,8 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { // Verify that running a task reduces the cap sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, 3))) - sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( - 0L, BlockManagerId("executor-1", "host1", 1), 100L)) + sc.listenerBus.postToAll(SparkListenerExecutorAdded( + 0L, "executor-1", new ExecutorInfo("host1", 1))) sc.listenerBus.postToAll(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) assert(numExecutorsPending(manager) === 4) assert(addExecutors(manager) === 1) @@ -578,30 +578,28 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(removeTimes(manager).isEmpty) // New executors have registered - sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( - 0L, BlockManagerId("executor-1", "host1", 1), 100L)) + sc.listenerBus.postToAll(SparkListenerExecutorAdded( + 0L, "executor-1", new ExecutorInfo("host1", 1))) assert(executorIds(manager).size === 1) assert(executorIds(manager).contains("executor-1")) assert(removeTimes(manager).size === 1) assert(removeTimes(manager).contains("executor-1")) - sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( - 0L, BlockManagerId("executor-2", "host2", 1), 100L)) + sc.listenerBus.postToAll(SparkListenerExecutorAdded( + 0L, "executor-2", new ExecutorInfo("host2", 1))) assert(executorIds(manager).size === 2) assert(executorIds(manager).contains("executor-2")) assert(removeTimes(manager).size === 2) assert(removeTimes(manager).contains("executor-2")) // Existing executors have disconnected - sc.listenerBus.postToAll(SparkListenerBlockManagerRemoved( - 0L, BlockManagerId("executor-1", "host1", 1))) + sc.listenerBus.postToAll(SparkListenerExecutorRemoved(0L, "executor-1", "")) assert(executorIds(manager).size === 1) assert(!executorIds(manager).contains("executor-1")) assert(removeTimes(manager).size === 1) assert(!removeTimes(manager).contains("executor-1")) // Unknown executor has disconnected - sc.listenerBus.postToAll(SparkListenerBlockManagerRemoved( - 0L, BlockManagerId("executor-3", "host3", 1))) + sc.listenerBus.postToAll(SparkListenerExecutorRemoved(0L, "executor-3", "")) assert(executorIds(manager).size === 1) assert(removeTimes(manager).size === 1) } @@ -613,8 +611,8 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(removeTimes(manager).isEmpty) sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) - sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( - 0L, BlockManagerId("executor-1", "host1", 1), 100L)) + sc.listenerBus.postToAll(SparkListenerExecutorAdded( + 0L, "executor-1", new ExecutorInfo("host1", 1))) assert(executorIds(manager).size === 1) assert(executorIds(manager).contains("executor-1")) assert(removeTimes(manager).size === 0) @@ -625,16 +623,16 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { val manager = sc.executorAllocationManager.get assert(executorIds(manager).isEmpty) assert(removeTimes(manager).isEmpty) - sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( - 0L, BlockManagerId("executor-1", "host1", 1), 100L)) + sc.listenerBus.postToAll(SparkListenerExecutorAdded( + 0L, "executor-1", new ExecutorInfo("host1", 1))) sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) assert(executorIds(manager).size === 1) assert(executorIds(manager).contains("executor-1")) assert(removeTimes(manager).size === 0) - sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( - 0L, BlockManagerId("executor-2", "host1", 1), 100L)) + sc.listenerBus.postToAll(SparkListenerExecutorAdded( + 0L, "executor-2", new ExecutorInfo("host1", 1))) assert(executorIds(manager).size === 2) assert(executorIds(manager).contains("executor-2")) assert(removeTimes(manager).size === 1) From 4cdb26c174e479a144950d12e1ad180f361af1fd Mon Sep 17 00:00:00 2001 From: Makoto Fukuhara Date: Fri, 6 Feb 2015 11:11:38 -0800 Subject: [PATCH 123/578] [SPARK-5618][Spark Core][Minor] Optimise utility code. Author: Makoto Fukuhara Closes #4396 from fukuo33/fix-unnecessary-regex and squashes the following commits: cd07fd6 [Makoto Fukuhara] fix unnecessary regex. --- 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 72d15e65bcde6..61d287ca9c3ac 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1162,9 +1162,9 @@ private[spark] object Utils extends Logging { // finding the call site of a method. val SPARK_CORE_CLASS_REGEX = """^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?(\.broadcast)?\.[A-Z]""".r - val SCALA_CLASS_REGEX = """^scala""".r + val SCALA_CORE_CLASS_PREFIX = "scala" val isSparkCoreClass = SPARK_CORE_CLASS_REGEX.findFirstIn(className).isDefined - val isScalaClass = SCALA_CLASS_REGEX.findFirstIn(className).isDefined + val isScalaClass = className.startsWith(SCALA_CORE_CLASS_PREFIX) // If the class is a Spark internal class or a Scala class, then exclude. isSparkCoreClass || isScalaClass } From 32e964c410e7083b43264c46291e93cd206a8038 Mon Sep 17 00:00:00 2001 From: Kostas Sakellis Date: Fri, 6 Feb 2015 11:13:00 -0800 Subject: [PATCH 124/578] SPARK-2450 Adds executor log links to Web UI Adds links to stderr/stdout in the executor tab of the webUI for: 1) Standalone 2) Yarn client 3) Yarn cluster This tries to add the log url support in a general way so as to make it easy to add support for all the cluster managers. This is done by using environment variables to pass to the executor the log urls. The SPARK_LOG_URL_ prefix is used and so additional logs besides stderr/stdout can also be added. To propagate this information to the UI we use the onExecutorAdded spark listener event. Although this commit doesn't add log urls when running on a mesos cluster, it should be possible to add using the same mechanism. Author: Kostas Sakellis Author: Josh Rosen Closes #3486 from ksakellis/kostas-spark-2450 and squashes the following commits: d190936 [Josh Rosen] Fix a few minor style / formatting nits. Reset listener after each test Don't null listener out at end of main(). 8673fe1 [Kostas Sakellis] CR feedback. Hide the log column if there are no logs available 5bf6952 [Kostas Sakellis] [SPARK-2450] [CORE] Adds exeuctor log links to Web UI --- .../spark/deploy/worker/ExecutorRunner.scala | 7 +++ .../apache/spark/deploy/worker/Worker.scala | 1 + .../CoarseGrainedExecutorBackend.scala | 8 ++- .../cluster/CoarseGrainedClusterMessage.scala | 6 +- .../CoarseGrainedSchedulerBackend.scala | 4 +- .../scheduler/cluster/ExecutorData.scala | 5 +- .../scheduler/cluster/ExecutorInfo.scala | 9 +-- .../cluster/mesos/MesosSchedulerBackend.scala | 3 +- .../apache/spark/ui/exec/ExecutorsPage.scala | 31 ++++++++-- .../apache/spark/ui/exec/ExecutorsTab.scala | 6 ++ .../org/apache/spark/util/JsonProtocol.scala | 6 +- .../spark/deploy/JsonProtocolSuite.scala | 2 +- .../spark/deploy/LogUrlsStandaloneSuite.scala | 59 +++++++++++++++++++ .../deploy/worker/ExecutorRunnerTest.scala | 2 +- .../mesos/MesosSchedulerBackendSuite.scala | 4 +- .../apache/spark/util/JsonProtocolSuite.scala | 12 +++- .../spark/deploy/yarn/ExecutorRunnable.scala | 12 +++- .../spark/deploy/yarn/YarnClusterSuite.scala | 31 +++++++++- 18 files changed, 178 insertions(+), 30 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala 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 bc9f78b9e5c77..0add3064da452 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 @@ -43,6 +43,7 @@ private[spark] class ExecutorRunner( val worker: ActorRef, val workerId: String, val host: String, + val webUiPort: Int, val sparkHome: File, val executorDir: File, val workerUrl: String, @@ -134,6 +135,12 @@ private[spark] class ExecutorRunner( // 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") + + // Add webUI log urls + val baseUrl = s"http://$host:$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") + process = builder.start() val header = "Spark Executor Command: %s\n%s\n\n".format( command.mkString("\"", "\" \"", "\""), "=" * 40) 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 b20f5c0c82895..10929eb516041 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 @@ -362,6 +362,7 @@ private[spark] class Worker( self, workerId, host, + webUiPort, sparkHome, executorDir, akkaUrl, 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 bc72c8970319c..3a42f8b157977 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -49,10 +49,16 @@ private[spark] class CoarseGrainedExecutorBackend( override def preStart() { logInfo("Connecting to driver: " + driverUrl) driver = context.actorSelection(driverUrl) - driver ! RegisterExecutor(executorId, hostPort, cores) + driver ! RegisterExecutor(executorId, hostPort, cores, extractLogUrls) context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) } + def extractLogUrls: Map[String, String] = { + val prefix = "SPARK_LOG_URL_" + sys.env.filterKeys(_.startsWith(prefix)) + .map(e => (e._1.substring(prefix.length).toLowerCase, e._2)) + } + override def receiveWithLogging = { case RegisteredExecutor => logInfo("Successfully registered with driver") 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 1da6fe976da5b..9bf74f4be198d 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 @@ -39,7 +39,11 @@ private[spark] object CoarseGrainedClusterMessages { case class RegisterExecutorFailed(message: String) extends CoarseGrainedClusterMessage // Executors to driver - case class RegisterExecutor(executorId: String, hostPort: String, cores: Int) + case class RegisterExecutor( + executorId: String, + hostPort: String, + cores: Int, + logUrls: Map[String, String]) extends CoarseGrainedClusterMessage { Utils.checkHostPort(hostPort, "Expected host port") } 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 103a5c053c289..9d2fb4f3b4729 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 @@ -86,7 +86,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste } def receiveWithLogging = { - case RegisterExecutor(executorId, hostPort, cores) => + case RegisterExecutor(executorId, hostPort, cores, logUrls) => Utils.checkHostPort(hostPort, "Host port expected " + hostPort) if (executorDataMap.contains(executorId)) { sender ! RegisterExecutorFailed("Duplicate executor ID: " + executorId) @@ -98,7 +98,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) val (host, _) = Utils.parseHostPort(hostPort) - val data = new ExecutorData(sender, sender.path.address, host, cores, cores) + 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 { 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 eb52ddfb1eab1..5e571efe76720 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 @@ -33,5 +33,6 @@ private[cluster] class ExecutorData( val executorAddress: Address, override val executorHost: String, var freeCores: Int, - override val totalCores: Int -) extends ExecutorInfo(executorHost, totalCores) + override val totalCores: Int, + override val logUrlMap: Map[String, String] +) extends ExecutorInfo(executorHost, totalCores, logUrlMap) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala index b4738e64c9391..7f218566146a1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala @@ -25,8 +25,8 @@ import org.apache.spark.annotation.DeveloperApi @DeveloperApi class ExecutorInfo( val executorHost: String, - val totalCores: Int -) { + val totalCores: Int, + val logUrlMap: Map[String, String]) { def canEqual(other: Any): Boolean = other.isInstanceOf[ExecutorInfo] @@ -34,12 +34,13 @@ class ExecutorInfo( case that: ExecutorInfo => (that canEqual this) && executorHost == that.executorHost && - totalCores == that.totalCores + totalCores == that.totalCores && + logUrlMap == that.logUrlMap case _ => false } override def hashCode(): Int = { - val state = Seq(executorHost, totalCores) + val state = Seq(executorHost, totalCores, logUrlMap) state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b) } } 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 c3c546be6da15..cfb6592e14aa8 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 @@ -270,7 +270,8 @@ private[spark] class MesosSchedulerBackend( mesosTasks.foreach { case (slaveId, tasks) => slaveIdToWorkerOffer.get(slaveId).foreach(o => listenerBus.post(SparkListenerExecutorAdded(System.currentTimeMillis(), slaveId, - new ExecutorInfo(o.host, o.cores))) + // TODO: Add support for log urls for Mesos + new ExecutorInfo(o.host, o.cores, Map.empty))) ) d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters) } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 363cb96de7998..956608d7c0cbe 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -26,7 +26,8 @@ import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage} import org.apache.spark.util.Utils /** Summary information about an executor to display in the UI. */ -private case class ExecutorSummaryInfo( +// Needs to be private[ui] because of a false positive MiMa failure. +private[ui] case class ExecutorSummaryInfo( id: String, hostPort: String, rddBlocks: Int, @@ -40,7 +41,8 @@ private case class ExecutorSummaryInfo( totalInputBytes: Long, totalShuffleRead: Long, totalShuffleWrite: Long, - maxMemory: Long) + maxMemory: Long, + executorLogs: Map[String, String]) private[ui] class ExecutorsPage( parent: ExecutorsTab, @@ -55,6 +57,7 @@ private[ui] class ExecutorsPage( val diskUsed = storageStatusList.map(_.diskUsed).sum val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) val execInfoSorted = execInfo.sortBy(_.id) + val logsExist = execInfo.filter(_.executorLogs.nonEmpty).nonEmpty val execTable = @@ -79,10 +82,11 @@ private[ui] class ExecutorsPage( Shuffle Write + {if (logsExist) else Seq.empty} {if (threadDumpEnabled) else Seq.empty} - {execInfoSorted.map(execRow)} + {execInfoSorted.map(execRow(_, logsExist))}
LogsThread Dump
@@ -107,7 +111,7 @@ private[ui] class ExecutorsPage( } /** Render an HTML row representing an executor */ - private def execRow(info: ExecutorSummaryInfo): Seq[Node] = { + private def execRow(info: ExecutorSummaryInfo, logsExist: Boolean): Seq[Node] = { val maximumMemory = info.maxMemory val memoryUsed = info.memoryUsed val diskUsed = info.diskUsed @@ -138,6 +142,21 @@ private[ui] class ExecutorsPage( {Utils.bytesToString(info.totalShuffleWrite)} + { + if (logsExist) { + + { + info.executorLogs.map { case (logName, logUrl) => + + } + } + + } + } { if (threadDumpEnabled) { val encodedId = URLEncoder.encode(info.id, "UTF-8") @@ -168,6 +187,7 @@ private[ui] class ExecutorsPage( val totalInputBytes = listener.executorToInputBytes.getOrElse(execId, 0L) val totalShuffleRead = listener.executorToShuffleRead.getOrElse(execId, 0L) val totalShuffleWrite = listener.executorToShuffleWrite.getOrElse(execId, 0L) + val executorLogs = listener.executorToLogUrls.getOrElse(execId, Map.empty) new ExecutorSummaryInfo( execId, @@ -183,7 +203,8 @@ private[ui] class ExecutorsPage( totalInputBytes, totalShuffleRead, totalShuffleWrite, - maxMem + maxMem, + executorLogs ) } } 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 dd1c2b78c4094..a38cb75fdd8c6 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 @@ -51,9 +51,15 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) extends Sp val executorToOutputBytes = HashMap[String, Long]() val executorToShuffleRead = HashMap[String, Long]() val executorToShuffleWrite = HashMap[String, Long]() + val executorToLogUrls = HashMap[String, Map[String, String]]() def storageStatusList = storageStatusListener.storageStatusList + override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded) = synchronized { + val eid = executorAdded.executorId + executorToLogUrls(eid) = executorAdded.executorInfo.logUrlMap + } + override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { val eid = taskStart.taskInfo.executorId executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 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 8e0e41ad3782e..c8407bbcb780b 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -383,7 +383,8 @@ private[spark] object JsonProtocol { def executorInfoToJson(executorInfo: ExecutorInfo): JValue = { ("Host" -> executorInfo.executorHost) ~ - ("Total Cores" -> executorInfo.totalCores) + ("Total Cores" -> executorInfo.totalCores) ~ + ("Log Urls" -> mapToJson(executorInfo.logUrlMap)) } /** ------------------------------ * @@ -792,7 +793,8 @@ private[spark] object JsonProtocol { def executorInfoFromJson(json: JValue): ExecutorInfo = { val executorHost = (json \ "Host").extract[String] val totalCores = (json \ "Total Cores").extract[Int] - new ExecutorInfo(executorHost, totalCores) + val logUrls = mapFromJson(json \ "Log Urls").toMap + new ExecutorInfo(executorHost, totalCores, logUrls) } /** -------------------------------- * 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 aa65f7e8915e6..ed02ca81e405c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -117,7 +117,7 @@ class JsonProtocolSuite extends FunSuite { } def createExecutorRunner(): ExecutorRunner = { - new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host", + new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host", 123, 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 new file mode 100644 index 0000000000000..f33bdc73e40ac --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.mutable + +import org.scalatest.{BeforeAndAfter, FunSuite} + +import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.scheduler.{SparkListenerExecutorAdded, SparkListener} +import org.apache.spark.{SparkContext, LocalSparkContext} + +class LogUrlsStandaloneSuite extends FunSuite with LocalSparkContext with BeforeAndAfter { + + /** Length of time to wait while draining listener events. */ + val WAIT_TIMEOUT_MILLIS = 10000 + + before { + sc = new SparkContext("local-cluster[2,1,512]", "test") + } + + test("verify log urls get propagated from workers") { + 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() + + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + listener.addedExecutorInfos.values.foreach { info => + assert(info.logUrlMap.nonEmpty) + } + } + + private class SaveExecutorInfo extends SparkListener { + val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() + + override def onExecutorAdded(executor: SparkListenerExecutorAdded) { + addedExecutorInfos(executor.executorId) = executor.executorInfo + } + } +} 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 6f233d7cf97aa..76511699e5ac5 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 @@ -32,7 +32,7 @@ class ExecutorRunnerTest extends FunSuite { val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) 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", + val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", 123, new File(sparkHome), new File("ooga"), "blah", new SparkConf, Seq("localDir"), ExecutorState.RUNNING) val builder = CommandUtils.buildProcessBuilder(appDesc.command, 512, sparkHome, er.substituteVariables) 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 f2ff98eb72daf..46ab02bfef780 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 @@ -43,7 +43,7 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea conf.set("spark.mesos.executor.home" , "/mesos-home") val listenerBus = EasyMock.createMock(classOf[LiveListenerBus]) - listenerBus.post(SparkListenerExecutorAdded(EasyMock.anyLong, "s1", new ExecutorInfo("host1", 2))) + listenerBus.post(SparkListenerExecutorAdded(EasyMock.anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) EasyMock.replay(listenerBus) val sc = EasyMock.createMock(classOf[SparkContext]) @@ -88,7 +88,7 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea val taskScheduler = EasyMock.createMock(classOf[TaskSchedulerImpl]) val listenerBus = EasyMock.createMock(classOf[LiveListenerBus]) - listenerBus.post(SparkListenerExecutorAdded(EasyMock.anyLong, "s1", new ExecutorInfo("host1", 2))) + listenerBus.post(SparkListenerExecutorAdded(EasyMock.anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) EasyMock.replay(listenerBus) val sc = EasyMock.createMock(classOf[SparkContext]) 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 6577ebaa2e9a8..842f54529baf0 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -76,8 +76,9 @@ class JsonProtocolSuite extends FunSuite { val unpersistRdd = SparkListenerUnpersistRDD(12345) val applicationStart = SparkListenerApplicationStart("The winner of all", None, 42L, "Garfield") val applicationEnd = SparkListenerApplicationEnd(42L) + val logUrlMap = Map("stderr" -> "mystderr", "stdout" -> "mystdout").toMap val executorAdded = SparkListenerExecutorAdded(executorAddedTime, "exec1", - new ExecutorInfo("Hostee.awesome.com", 11)) + new ExecutorInfo("Hostee.awesome.com", 11, logUrlMap)) val executorRemoved = SparkListenerExecutorRemoved(executorRemovedTime, "exec2", "test reason") testEvent(stageSubmitted, stageSubmittedJsonString) @@ -100,13 +101,14 @@ class JsonProtocolSuite extends FunSuite { } test("Dependent Classes") { + val logUrlMap = Map("stderr" -> "mystderr", "stdout" -> "mystdout").toMap testRDDInfo(makeRddInfo(2, 3, 4, 5L, 6L)) testStageInfo(makeStageInfo(10, 20, 30, 40L, 50L)) testTaskInfo(makeTaskInfo(999L, 888, 55, 777L, false)) testTaskMetrics(makeTaskMetrics( 33333L, 44444L, 55555L, 66666L, 7, 8, hasHadoopInput = false, hasOutput = false)) testBlockManagerId(BlockManagerId("Hong", "Kong", 500)) - testExecutorInfo(new ExecutorInfo("host", 43)) + testExecutorInfo(new ExecutorInfo("host", 43, logUrlMap)) // StorageLevel testStorageLevel(StorageLevel.NONE) @@ -1463,7 +1465,11 @@ class JsonProtocolSuite extends FunSuite { | "Executor ID": "exec1", | "Executor Info": { | "Host": "Hostee.awesome.com", - | "Total Cores": 11 + | "Total Cores": 11, + | "Log Urls" : { + | "stderr" : "mystderr", + | "stdout" : "mystdout" + | } | } |} """ 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 ee2002a35f523..408cf09b9bdfa 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 @@ -56,7 +56,7 @@ class ExecutorRunnable( var rpc: YarnRPC = YarnRPC.create(conf) var nmClient: NMClient = _ val yarnConf: YarnConfiguration = new YarnConfiguration(conf) - lazy val env = prepareEnvironment + lazy val env = prepareEnvironment(container) def run = { logInfo("Starting Executor Container") @@ -254,7 +254,7 @@ class ExecutorRunnable( localResources } - private def prepareEnvironment: HashMap[String, String] = { + 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) @@ -270,6 +270,14 @@ class ExecutorRunnable( YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) } + // 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) + env("SPARK_LOG_URL_STDERR") = s"$baseUrl/stderr?start=0" + env("SPARK_LOG_URL_STDOUT") = s"$baseUrl/stdout?start=0" + } + System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k, v) => env(k) = v } env } 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 7165918e1bfcf..eda40efc4c77f 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 @@ -21,16 +21,17 @@ import java.io.File import java.util.concurrent.TimeUnit import scala.collection.JavaConversions._ +import scala.collection.mutable import com.google.common.base.Charsets import com.google.common.io.Files -import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers} - 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.deploy.SparkHadoopUtil +import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorAdded} import org.apache.spark.util.Utils class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers with Logging { @@ -143,6 +144,11 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit 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) + } } test("run Spark in yarn-cluster mode") { @@ -156,6 +162,11 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit "--num-executors", "1") Client.main(args) checkResult(result) + + // verify log urls are present. + YarnClusterDriver.listener.addedExecutorInfos.values.foreach { info => + assert(info.logUrlMap.nonEmpty) + } } test("run Spark in yarn-cluster mode unsuccessfully") { @@ -203,8 +214,19 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit } +private class SaveExecutorInfo extends SparkListener { + val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() + + override def onExecutorAdded(executor : SparkListenerExecutorAdded) { + addedExecutorInfos(executor.executorId) = executor.executorInfo + } +} + 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) { System.err.println( @@ -216,12 +238,15 @@ private object YarnClusterDriver extends Logging with Matchers { System.exit(1) } + listener = new SaveExecutorInfo val sc = new SparkContext(new SparkConf().setMaster(args(0)) .setAppName("yarn \"test app\" 'with quotes' and \\back\\slashes and $dollarSigns")) + sc.addSparkListener(listener) val status = new File(args(1)) 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 { From 0d74bd7fd7b2722d08eddc5c269b8b2b6cb47635 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 6 Feb 2015 11:14:58 -0800 Subject: [PATCH 125/578] [SPARK-5555] Enable UISeleniumSuite tests This patch enables UISeleniumSuite, a set of tests for the Spark application web UI. These tests were previously disabled because they were slow, but I think we now have sufficient test time budget that the benefit of enabling them outweighs the time costs. Author: Josh Rosen Closes #4334 from JoshRosen/enable-uiseleniumsuite and squashes the following commits: 4ab9477 [Josh Rosen] Use BeforeAndAfterAll to cleanup WebDriver 71efc72 [Josh Rosen] Update broken UISeleniumSuite tests; use random port #. a5ab595 [Josh Rosen] Enable UISeleniumSuite tests. --- .../org/apache/spark/ui/UISeleniumSuite.scala | 29 +++++++++++++------ 1 file changed, 20 insertions(+), 9 deletions(-) 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 e85a436cdba17..6a972381faf14 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -32,12 +32,21 @@ import org.apache.spark.api.java.StorageLevels import org.apache.spark.shuffle.FetchFailedException /** - * Selenium tests for the Spark Web UI. These tests are not run by default - * because they're slow. + * Selenium tests for the Spark Web UI. */ -@DoNotDiscover -class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { - implicit val webDriver: WebDriver = new HtmlUnitDriver +class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with BeforeAndAfterAll { + + implicit var webDriver: WebDriver = _ + + override def beforeAll(): Unit = { + webDriver = new HtmlUnitDriver + } + + override def afterAll(): Unit = { + if (webDriver != null) { + webDriver.quit() + } + } /** * Create a test SparkContext with the SparkUI enabled. @@ -48,6 +57,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { .setMaster("local") .setAppName("test") .set("spark.ui.enabled", "true") + .set("spark.ui.port", "0") val sc = new SparkContext(conf) assert(sc.ui.isDefined) sc @@ -93,7 +103,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { } eventually(timeout(5 seconds), interval(50 milliseconds)) { go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") - find(id("active")).get.text should be("Active Stages (0)") + find(id("active")) should be(None) // Since we hide empty tables find(id("failed")).get.text should be("Failed Stages (1)") } @@ -105,7 +115,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { } eventually(timeout(5 seconds), interval(50 milliseconds)) { go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") - find(id("active")).get.text should be("Active Stages (0)") + find(id("active")) should be(None) // Since we hide empty tables // The failure occurs before the stage becomes active, hence we should still show only one // failed stage, not two: find(id("failed")).get.text should be("Failed Stages (1)") @@ -167,13 +177,14 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { test("job progress bars should handle stage / task failures") { withSpark(newSparkContext()) { sc => - val data = sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity) + val data = sc.parallelize(Seq(1, 2, 3), 1).map(identity).groupBy(identity) val shuffleHandle = data.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle // Simulate fetch failures: val mappedData = data.map { x => val taskContext = TaskContext.get - if (taskContext.attemptNumber == 0) { // Cause this stage to fail on its first attempt. + if (taskContext.taskAttemptId() == 1) { + // Cause the post-shuffle stage to fail on its first attempt with a single task failure val env = SparkEnv.get val bmAddress = env.blockManager.blockManagerId val shuffleId = shuffleHandle.shuffleId From 80f3bcb58f836cfe1829c85bdd349c10525c8a5e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 6 Feb 2015 11:22:11 -0800 Subject: [PATCH 126/578] [SPARK-5652][Mllib] Use broadcasted weights in LogisticRegressionModel `LogisticRegressionModel`'s `predictPoint` should directly use broadcasted weights. This pr also fixes the compilation errors of two unit test suite: `JavaLogisticRegressionSuite ` and `JavaLinearRegressionSuite`. Author: Liang-Chi Hsieh Closes #4429 from viirya/use_bcvalue and squashes the following commits: 5a797e5 [Liang-Chi Hsieh] Use broadcasted weights. Fix compilation error. --- .../spark/mllib/classification/LogisticRegression.scala | 8 ++++---- .../ml/classification/JavaLogisticRegressionSuite.java | 4 ++-- .../spark/ml/regression/JavaLinearRegressionSuite.java | 4 ++-- 3 files changed, 8 insertions(+), 8 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 a668e7a7a359f..9a391bfff76a3 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 @@ -33,7 +33,7 @@ import org.apache.spark.rdd.RDD * * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. (Only used in Binary Logistic Regression. - * In Multinomial Logistic Regression, the intercepts will not be a single values, + * In Multinomial Logistic Regression, the intercepts will not be a single value, * so the intercepts will be part of the weights.) * @param numFeatures the dimension of the features. * @param numClasses the number of possible outcomes for k classes classification problem in @@ -107,7 +107,7 @@ 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(weights, dataMatrix) + intercept + val margin = dot(weightMatrix, dataMatrix) + intercept val score = 1.0 / (1.0 + math.exp(-margin)) threshold match { case Some(t) => if (score > t) 1.0 else 0.0 @@ -116,11 +116,11 @@ class LogisticRegressionModel ( } else { val dataWithBiasSize = weightMatrix.size / (numClasses - 1) - val weightsArray = weights match { + val weightsArray = weightMatrix match { case dv: DenseVector => dv.values case _ => throw new IllegalArgumentException( - s"weights only supports dense vector but got type ${weights.getClass}.") + s"weights only supports dense vector but got type ${weightMatrix.getClass}.") } val margins = (0 until numClasses - 1).map { i => 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 26284023b0f69..d4b664479255d 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 @@ -84,7 +84,7 @@ public void logisticRegressionWithSetters() { .setThreshold(0.6) .setProbabilityCol("myProbability"); LogisticRegressionModel model = lr.fit(dataset); - assert(model.fittingParamMap().apply(lr.maxIter()) == 10); + assert(model.fittingParamMap().apply(lr.maxIter()).equals(10)); assert(model.fittingParamMap().apply(lr.regParam()).equals(1.0)); assert(model.fittingParamMap().apply(lr.threshold()).equals(0.6)); assert(model.getThreshold() == 0.6); @@ -109,7 +109,7 @@ public void logisticRegressionWithSetters() { // Call fit() with new params, and check as many params as we can. LogisticRegressionModel model2 = lr.fit(dataset, lr.maxIter().w(5), lr.regParam().w(0.1), lr.threshold().w(0.4), lr.probabilityCol().w("theProb")); - assert(model2.fittingParamMap().apply(lr.maxIter()) == 5); + assert(model2.fittingParamMap().apply(lr.maxIter()).equals(5)); assert(model2.fittingParamMap().apply(lr.regParam()).equals(0.1)); assert(model2.fittingParamMap().apply(lr.threshold()).equals(0.4)); assert(model2.getThreshold() == 0.4); 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 5bd616e74d86c..40d5a92bb32af 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 @@ -76,13 +76,13 @@ public void linearRegressionWithSetters() { .setMaxIter(10) .setRegParam(1.0); LinearRegressionModel model = lr.fit(dataset); - assert(model.fittingParamMap().apply(lr.maxIter()) == 10); + assert(model.fittingParamMap().apply(lr.maxIter()).equals(10)); assert(model.fittingParamMap().apply(lr.regParam()).equals(1.0)); // Call fit() with new params, and check as many params as we can. LinearRegressionModel model2 = lr.fit(dataset, lr.maxIter().w(5), lr.regParam().w(0.1), lr.predictionCol().w("thePred")); - assert(model2.fittingParamMap().apply(lr.maxIter()) == 5); + assert(model2.fittingParamMap().apply(lr.maxIter()).equals(5)); assert(model2.fittingParamMap().apply(lr.regParam()).equals(0.1)); assert(model2.getPredictionCol().equals("thePred")); } From fb6c0cbac414f3c43dec2ab886ca8a1097b781f7 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 6 Feb 2015 11:47:32 -0800 Subject: [PATCH 127/578] [HOTFIX] Fix test build break in ExecutorAllocationManagerSuite. This was caused because #3486 added a new field to ExecutorInfo and #4369 added new tests that created ExecutorInfos. These patches were merged in quick succession and were never tested together, hence the compilation error. --- .../spark/ExecutorAllocationManagerSuite.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index c2869628afbb3..9eb87f016068d 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -145,7 +145,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { // Verify that running a task reduces the cap sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, 3))) sc.listenerBus.postToAll(SparkListenerExecutorAdded( - 0L, "executor-1", new ExecutorInfo("host1", 1))) + 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) sc.listenerBus.postToAll(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) assert(numExecutorsPending(manager) === 4) assert(addExecutors(manager) === 1) @@ -579,13 +579,13 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { // New executors have registered sc.listenerBus.postToAll(SparkListenerExecutorAdded( - 0L, "executor-1", new ExecutorInfo("host1", 1))) + 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) assert(executorIds(manager).size === 1) assert(executorIds(manager).contains("executor-1")) assert(removeTimes(manager).size === 1) assert(removeTimes(manager).contains("executor-1")) sc.listenerBus.postToAll(SparkListenerExecutorAdded( - 0L, "executor-2", new ExecutorInfo("host2", 1))) + 0L, "executor-2", new ExecutorInfo("host2", 1, Map.empty))) assert(executorIds(manager).size === 2) assert(executorIds(manager).contains("executor-2")) assert(removeTimes(manager).size === 2) @@ -612,7 +612,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) sc.listenerBus.postToAll(SparkListenerExecutorAdded( - 0L, "executor-1", new ExecutorInfo("host1", 1))) + 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) assert(executorIds(manager).size === 1) assert(executorIds(manager).contains("executor-1")) assert(removeTimes(manager).size === 0) @@ -624,7 +624,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(executorIds(manager).isEmpty) assert(removeTimes(manager).isEmpty) sc.listenerBus.postToAll(SparkListenerExecutorAdded( - 0L, "executor-1", new ExecutorInfo("host1", 1))) + 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) assert(executorIds(manager).size === 1) @@ -632,7 +632,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(removeTimes(manager).size === 0) sc.listenerBus.postToAll(SparkListenerExecutorAdded( - 0L, "executor-2", new ExecutorInfo("host1", 1))) + 0L, "executor-2", new ExecutorInfo("host1", 1, Map.empty))) assert(executorIds(manager).size === 2) assert(executorIds(manager).contains("executor-2")) assert(removeTimes(manager).size === 1) From af2a2a263ac5d890e84d012b75fcb50e02c9ede8 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Fri, 6 Feb 2015 11:50:20 -0800 Subject: [PATCH 128/578] [SPARK-4361][Doc] Add more docs for Hadoop Configuration I'm trying to point out reusing a Configuration in these APIs is dangerous. Any better idea? Author: zsxwing Closes #3225 from zsxwing/SPARK-4361 and squashes the following commits: fe4e3d5 [zsxwing] Add more docs for Hadoop Configuration --- .../scala/org/apache/spark/SparkContext.scala | 20 +++++++++++-- .../spark/api/java/JavaSparkContext.scala | 28 +++++++++++++++++++ 2 files changed, 46 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 24490fddc5c6a..5623587c36fa6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -288,7 +288,12 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // the bound port to the cluster manager properly ui.foreach(_.bind()) - /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ + /** + * A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. + * + * '''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) // Add each JAR given through the constructor @@ -694,7 +699,10 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable), * using the older MapReduce API (`org.apache.hadoop.mapred`). * - * @param conf JobConf for setting up the dataset + * @param conf JobConf for setting up the dataset. Note: This will be put into a Broadcast. + * Therefore if you plan to reuse this conf to create multiple RDDs, you need to make + * sure you won't modify the conf. A safe approach is always creating a new conf for + * a new RDD. * @param inputFormatClass Class of the InputFormat * @param keyClass Class of the keys * @param valueClass Class of the values @@ -830,6 +838,14 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * Get an RDD for a given Hadoop file with an arbitrary new API InputFormat * and extra configuration options to pass to the input format. * + * @param conf Configuration for setting up the dataset. Note: This will be put into a Broadcast. + * Therefore if you plan to reuse this conf to create multiple RDDs, you need to make + * sure you won't modify the conf. A safe approach is always creating a new conf for + * a new RDD. + * @param fClass Class of the InputFormat + * @param kClass Class of the keys + * @param vClass Class of the values + * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each * record, directly caching the returned RDD or directly passing it to an aggregation or shuffle * operation will create many references to the same object. 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 97f5c9f257e09..6d6ed693be752 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 @@ -373,6 +373,15 @@ class JavaSparkContext(val sc: SparkContext) * other necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable, * etc). * + * @param conf JobConf for setting up the dataset. Note: This will be put into a Broadcast. + * Therefore if you plan to reuse this conf to create multiple RDDs, you need to make + * sure you won't modify the conf. A safe approach is always creating a new conf for + * a new RDD. + * @param inputFormatClass Class of the InputFormat + * @param keyClass Class of the keys + * @param valueClass Class of the values + * @param minPartitions Minimum number of Hadoop Splits to generate. + * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each * record, directly caching the returned RDD will create many references to the same object. * If you plan to directly cache Hadoop writable objects, you should first copy them using @@ -395,6 +404,14 @@ class JavaSparkContext(val sc: SparkContext) * Get an RDD for a Hadoop-readable dataset from a Hadooop JobConf giving its InputFormat and any * other necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable, * + * @param conf JobConf for setting up the dataset. Note: This will be put into a Broadcast. + * Therefore if you plan to reuse this conf to create multiple RDDs, you need to make + * sure you won't modify the conf. A safe approach is always creating a new conf for + * a new RDD. + * @param inputFormatClass Class of the InputFormat + * @param keyClass Class of the keys + * @param valueClass Class of the values + * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each * record, directly caching the returned RDD will create many references to the same object. * If you plan to directly cache Hadoop writable objects, you should first copy them using @@ -476,6 +493,14 @@ class JavaSparkContext(val sc: SparkContext) * Get an RDD for a given Hadoop file with an arbitrary new API InputFormat * and extra configuration options to pass to the input format. * + * @param conf Configuration for setting up the dataset. Note: This will be put into a Broadcast. + * Therefore if you plan to reuse this conf to create multiple RDDs, you need to make + * sure you won't modify the conf. A safe approach is always creating a new conf for + * a new RDD. + * @param fClass Class of the InputFormat + * @param kClass Class of the keys + * @param vClass Class of the values + * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each * record, directly caching the returned RDD will create many references to the same object. * If you plan to directly cache Hadoop writable objects, you should first copy them using @@ -675,6 +700,9 @@ class JavaSparkContext(val sc: SparkContext) /** * Returns the Hadoop configuration used for the Hadoop code (e.g. file systems) we reuse. + * + * '''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. */ def hadoopConfiguration(): Configuration = { sc.hadoopConfiguration From d34f79c8db79ae461fadae190446ebc19091bec9 Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Fri, 6 Feb 2015 11:57:02 -0800 Subject: [PATCH 129/578] [SPARK-2945][YARN][Doc]add doc for spark.executor.instances https://issues.apache.org/jira/browse/SPARK-2945 spark.executor.instances works. As this JIRA recommended, we should add docs for this common config. Author: WangTaoTheTonic Closes #4350 from WangTaoTheTonic/SPARK-2945 and squashes the following commits: 4c3913a [WangTaoTheTonic] not compatible with dynamic allocation 5fa9c46 [WangTaoTheTonic] add doc for spark.executor.instances --- docs/running-on-yarn.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 68ab127bcf087..2b93eef6c26ed 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -104,6 +104,13 @@ Most of the configs are the same for Spark on YARN as for other deployment modes Comma-separated list of files to be placed in the working directory of each executor. + + spark.executor.instances + 2 + + The number of executors. Note that this property is incompatible with spark.dynamicAllocation.enabled. + + spark.yarn.executor.memoryOverhead executorMemory * 0.07, with minimum of 384 From 70e5b030a78ddfdcc8c9eee568009f277dee0872 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Fri, 6 Feb 2015 12:08:22 -0800 Subject: [PATCH 130/578] [SPARK-5628] Add version option to spark-ec2 Every proper command line tool should include a `--version` option or something similar. This PR adds this to `spark-ec2` using the standard functionality provided by `optparse`. One thing we don't do here is follow the Python convention of setting `__version__`, since it seems awkward given how `spark-ec2` is laid out. Author: Nicholas Chammas Closes #4414 from nchammas/spark-ec2-show-version and squashes the following commits: 914cab5 [Nicholas Chammas] add version info --- ec2/spark_ec2.py | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index abab209a05ba0..0de4a62e203fd 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -39,6 +39,9 @@ from optparse import OptionParser from sys import stderr +SPARK_EC2_VERSION = "1.2.0" +SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__)) + VALID_SPARK_VERSIONS = set([ "0.7.3", "0.8.0", @@ -54,9 +57,8 @@ "1.2.0", ]) -DEFAULT_SPARK_VERSION = "1.2.0" +DEFAULT_SPARK_VERSION = SPARK_EC2_VERSION DEFAULT_SPARK_GITHUB_REPO = "https://github.com/apache/spark" -SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__)) MESOS_SPARK_EC2_BRANCH = "branch-1.3" # A URL prefix from which to fetch AMI information @@ -103,12 +105,10 @@ class UsageError(Exception): # Configure and parse our command-line arguments def parse_args(): parser = OptionParser( - usage="spark-ec2 [options] " - + "\n\n can be: launch, destroy, login, stop, start, get-master, reboot-slaves", - add_help_option=False) - parser.add_option( - "-h", "--help", action="help", - help="Show this help message and exit") + prog="spark-ec2", + 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)") From d433816157bb3ae1f0fbe44efec43a0c906d9f82 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 6 Feb 2015 12:13:44 -0800 Subject: [PATCH 131/578] [SPARK-5650][SQL] Support optional 'FROM' clause In Hive, 'FROM' clause is optional. This pr supports it. Author: Liang-Chi Hsieh Closes #4426 from viirya/optional_from and squashes the following commits: fe81f31 [Liang-Chi Hsieh] Support optional 'FROM' clause. --- .../org/apache/spark/sql/hive/HiveQl.scala | 19 ++++++++++++++----- ... clause-0-b42b408a87b258921240058f880a721a | 1 + .../sql/hive/execution/HiveQuerySuite.scala | 3 +++ 3 files changed, 18 insertions(+), 5 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/no from clause-0-b42b408a87b258921240058f880a721a 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 c19a091719190..6265c61a1dabe 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 @@ -556,9 +556,14 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token("TOK_TRUNCATETABLE", Token("TOK_TABLE_PARTITION",table)::Nil) => NativePlaceholder - case Token("TOK_QUERY", - Token("TOK_FROM", fromClause :: Nil) :: - insertClauses) => + 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) + } // Return one query for each insert clause. val queries = insertClauses.map { case Token("TOK_INSERT", singleInsert) => @@ -599,8 +604,12 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C "TOK_LATERAL_VIEW"), singleInsert) } - - val relations = nodeToRelation(fromClause) + + val relations = fromClause match { + case Some(f) => nodeToRelation(f) + case None => NoRelation + } + val withWhere = whereClause.map { whereNode => val Seq(whereExpr) = whereNode.getChildren.toSeq Filter(nodeToExpr(whereExpr), relations) diff --git a/sql/hive/src/test/resources/golden/no from clause-0-b42b408a87b258921240058f880a721a b/sql/hive/src/test/resources/golden/no from clause-0-b42b408a87b258921240058f880a721a new file mode 100644 index 0000000000000..390d344ecb9d3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/no from clause-0-b42b408a87b258921240058f880a721a @@ -0,0 +1 @@ +1 1 -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 bb73ff1ea7e43..d16a1e0b73faa 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 @@ -202,6 +202,9 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("having no references", "SELECT key FROM src GROUP BY key HAVING COUNT(*) > 1") + createQueryTest("no from clause", + "SELECT 1, +1, -1") + createQueryTest("boolean = number", """ |SELECT From 500dc2b4b3136029457e708859fe27da93b1f9e8 Mon Sep 17 00:00:00 2001 From: Tobias Schlatter Date: Fri, 6 Feb 2015 12:15:02 -0800 Subject: [PATCH 132/578] [SPARK-5640] Synchronize ScalaReflection where necessary Author: Tobias Schlatter Closes #4431 from gzm0/sync-scala-refl and squashes the following commits: c5da21e [Tobias Schlatter] [SPARK-5640] Synchronize ScalaReflection where necessary --- .../org/apache/spark/sql/catalyst/ScalaReflection.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 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 0445f3aa07912..5d9c331ca5178 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 @@ -103,10 +103,11 @@ trait ScalaReflection { } /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ - def schemaFor[T: TypeTag]: Schema = schemaFor(typeOf[T]) + def schemaFor[T: TypeTag]: Schema = + ScalaReflectionLock.synchronized { schemaFor(typeOf[T]) } /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ - def schemaFor(tpe: `Type`): Schema = { + def schemaFor(tpe: `Type`): Schema = ScalaReflectionLock.synchronized { val className: String = tpe.erasure.typeSymbol.asClass.fullName tpe match { case t if Utils.classIsLoadable(className) && From a958d60975147fb1afc76fcbd80f65ac8d78759a Mon Sep 17 00:00:00 2001 From: q00251598 Date: Fri, 6 Feb 2015 12:29:26 -0800 Subject: [PATCH 133/578] [SPARK-5619][SQL] Support 'show roles' in HiveContext Author: q00251598 Closes #4397 from watermen/SPARK-5619 and squashes the following commits: f819b6c [q00251598] Support show roles in HiveContext. --- 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 6265c61a1dabe..be63aa1a93aee 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 @@ -68,6 +68,7 @@ private[hive] object HiveQl { "TOK_SHOWLOCKS", "TOK_UNLOCKTABLE", + "TOK_SHOW_ROLES", "TOK_CREATEROLE", "TOK_DROPROLE", "TOK_GRANT", From 0b7eb3f3b700080bf6cb810d092709a8a468e5db Mon Sep 17 00:00:00 2001 From: OopsOutOfMemory Date: Fri, 6 Feb 2015 12:33:20 -0800 Subject: [PATCH 134/578] [SPARK-5324][SQL] Results of describe can't be queried Make below code works. ``` sql("DESCRIBE test").registerTempTable("describeTest") sql("SELECT * FROM describeTest").collect() ``` Author: OopsOutOfMemory Author: Sheng, Li Closes #4249 from OopsOutOfMemory/desc_query and squashes the following commits: 6fee13d [OopsOutOfMemory] up-to-date e71430a [Sheng, Li] Update HiveOperatorQueryableSuite.scala 3ba1058 [OopsOutOfMemory] change to default argument aac7226 [OopsOutOfMemory] Merge branch 'master' into desc_query 68eb6dd [OopsOutOfMemory] Merge branch 'desc_query' of github.com:OopsOutOfMemory/spark into desc_query 354ad71 [OopsOutOfMemory] query describe command d541a35 [OopsOutOfMemory] refine test suite e1da481 [OopsOutOfMemory] refine test suite a780539 [OopsOutOfMemory] Merge branch 'desc_query' of github.com:OopsOutOfMemory/spark into desc_query 0015f82 [OopsOutOfMemory] code style dd0aaef [OopsOutOfMemory] code style c7d606d [OopsOutOfMemory] rename test suite 75f2342 [OopsOutOfMemory] refine code and test suite f942c9b [OopsOutOfMemory] initial 11559ae [OopsOutOfMemory] code style c5fdecf [OopsOutOfMemory] code style aeaea5f [OopsOutOfMemory] rename test suite ac2c3bb [OopsOutOfMemory] refine code and test suite 544573e [OopsOutOfMemory] initial --- .../org/apache/spark/sql/sources/ddl.scala | 11 ++-- .../org/apache/spark/sql/hive/HiveQl.scala | 5 +- .../HiveOperatorQueryableSuite.scala | 51 +++++++++++++++++++ .../sql/hive/execution/HiveQuerySuite.scala | 2 +- 4 files changed, 61 insertions(+), 8 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala 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 9c37e0169ff85..a692ef51b31ed 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 @@ -296,11 +296,14 @@ private[sql] case class ResolvedDataSource(provider: Class[_], relation: BaseRel private[sql] case class DescribeCommand( table: LogicalPlan, isExtended: Boolean) extends Command { - override def output = Seq( + override val output = Seq( // Column names are based on Hive. - AttributeReference("col_name", StringType, nullable = false)(), - AttributeReference("data_type", StringType, nullable = false)(), - AttributeReference("comment", StringType, nullable = false)()) + AttributeReference("col_name", StringType, nullable = false, + new MetadataBuilder().putString("comment", "name of the column").build())(), + AttributeReference("data_type", StringType, nullable = false, + new MetadataBuilder().putString("comment", "data type of the column").build())(), + AttributeReference("comment", StringType, nullable = false, + new MetadataBuilder().putString("comment", "comment of the column").build())()) } private[sql] case class CreateTableUsing( 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 be63aa1a93aee..4b7fa06532963 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 @@ -497,15 +497,14 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C // TODO: Actually, a user may mean tableName.columnName. Need to resolve this issue. val tableIdent = extractTableIdent(nameParts.head) DescribeCommand( - UnresolvedRelation(tableIdent, None), extended.isDefined) + UnresolvedRelation(tableIdent, None), isExtended = extended.isDefined) case Token(".", dbName :: tableName :: colName :: Nil) => // It is describing a column with the format like "describe db.table column". NativePlaceholder case tableName => // It is describing a table with the format like "describe table". DescribeCommand( - UnresolvedRelation(Seq(tableName.getText), None), - extended.isDefined) + UnresolvedRelation(Seq(tableName.getText), None), isExtended = extended.isDefined) } } // All other cases. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala new file mode 100644 index 0000000000000..efbef68cd4447 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.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.sql.hive.execution + +import org.apache.spark.sql.{Row, QueryTest} +import org.apache.spark.sql.hive.test.TestHive._ + +/** + * A set of tests that validates commands can also be queried by like a table + */ +class HiveOperatorQueryableSuite extends QueryTest { + test("SPARK-5324 query result of describe command") { + loadTestTable("src") + + // register a describe command to be a temp table + sql("desc src").registerTempTable("mydesc") + checkAnswer( + sql("desc mydesc"), + Seq( + Row("col_name", "string", "name of the column"), + Row("data_type", "string", "data type of the column"), + Row("comment", "string", "comment of the column"))) + + checkAnswer( + sql("select * from mydesc"), + Seq( + Row("key", "int", null), + Row("value", "string", null))) + + checkAnswer( + sql("select col_name, data_type, comment from mydesc"), + Seq( + Row("key", "int", null), + Row("value", "string", null))) + } +} 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 d16a1e0b73faa..27047ce4b1b0b 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 @@ -59,7 +59,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { Locale.setDefault(originalLocale) } - test("SPARK-4908: concurent hive native commands") { + test("SPARK-4908: concurrent hive native commands") { (1 to 100).par.map { _ => sql("USE default") sql("SHOW TABLES") From 3eccf29ce061559c86e6f7338851932fc89a9afd Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 6 Feb 2015 12:38:07 -0800 Subject: [PATCH 135/578] [SPARK-5595][SPARK-5603][SQL] Add a rule to do PreInsert type casting and field renaming and invalidating in memory cache after INSERT This PR adds a rule to Analyzer that will add preinsert data type casting and field renaming to the select clause in an `INSERT INTO/OVERWRITE` statement. Also, with the change of this PR, we always invalidate our in memory data cache after inserting into a BaseRelation. cc marmbrus liancheng Author: Yin Huai Closes #4373 from yhuai/insertFollowUp and squashes the following commits: 08237a7 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertFollowUp 316542e [Yin Huai] Doc update. c9ccfeb [Yin Huai] Revert a unnecessary change. 84aecc4 [Yin Huai] Address comments. 1951fe1 [Yin Huai] Merge remote-tracking branch 'upstream/master' c18da34 [Yin Huai] Invalidate cache after insert. 727f21a [Yin Huai] Preinsert casting and renaming. --- .../org/apache/spark/sql/SQLContext.scala | 6 +- .../apache/spark/sql/json/JSONRelation.scala | 12 ++- .../sql/sources/DataSourceStrategy.scala | 2 +- .../apache/spark/sql/sources/commands.scala | 10 ++- .../apache/spark/sql/sources/interfaces.scala | 16 ++++ .../org/apache/spark/sql/sources/rules.scala | 76 ++++++++++++++++++ .../org/apache/spark/sql/json/JsonSuite.scala | 25 ++++++ .../spark/sql/sources/DataSourceTest.scala | 6 +- .../spark/sql/sources/InsertIntoSuite.scala | 80 +++++++++++++++++++ .../apache/spark/sql/hive/HiveContext.scala | 1 + 10 files changed, 227 insertions(+), 7 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/sources/rules.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 706ef6ad4f174..bf3990671029e 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 @@ -91,7 +91,11 @@ class SQLContext(@transient val sparkContext: SparkContext) @transient protected[sql] lazy val analyzer: Analyzer = - new Analyzer(catalog, functionRegistry, caseSensitive = true) + new Analyzer(catalog, functionRegistry, caseSensitive = true) { + override val extendedRules = + sources.PreInsertCastAndRename :: + Nil + } @transient protected[sql] lazy val optimizer: Optimizer = DefaultOptimizer 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 f27585d05a986..c4e14c6c92908 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 @@ -72,7 +72,6 @@ private[sql] case class JSONRelation( userSpecifiedSchema: Option[StructType])( @transient val sqlContext: SQLContext) extends TableScan with InsertableRelation { - // TODO: Support partitioned JSON relation. private def baseRDD = sqlContext.sparkContext.textFile(path) @@ -99,10 +98,21 @@ private[sql] case class JSONRelation( s"Unable to clear output directory ${filesystemPath.toString} prior" + s" to INSERT OVERWRITE a JSON table:\n${e.toString}") } + // Write the data. data.toJSON.saveAsTextFile(path) + // Right now, we assume that the schema is not changed. We will not update the schema. + // schema = data.schema } else { // TODO: Support INSERT INTO sys.error("JSON table only support INSERT OVERWRITE for now.") } } + + override def hashCode(): Int = 41 * (41 + path.hashCode) + schema.hashCode() + + override def equals(other: Any): Boolean = other match { + case that: JSONRelation => + (this.path == that.path) && (this.schema == that.schema) + case _ => false + } } 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 d23ffb8b7a960..624369afe87b5 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 @@ -59,7 +59,7 @@ private[sql] object DataSourceStrategy extends Strategy { if (partition.nonEmpty) { sys.error(s"Insert into a partition is not allowed because $l is not partitioned.") } - execution.ExecutedCommand(InsertIntoRelation(t, query, overwrite)) :: Nil + execution.ExecutedCommand(InsertIntoDataSource(l, query, overwrite)) :: Nil case _ => Nil } 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 d7942dc30934b..c9cd0e6e93829 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,17 +19,21 @@ 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.RunnableCommand +import org.apache.spark.sql.execution.{LogicalRDD, RunnableCommand} -private[sql] case class InsertIntoRelation( - relation: InsertableRelation, +private[sql] case class InsertIntoDataSource( + logicalRelation: LogicalRelation, query: LogicalPlan, overwrite: Boolean) extends RunnableCommand { override def run(sqlContext: SQLContext) = { + val relation = logicalRelation.relation.asInstanceOf[InsertableRelation] relation.insert(DataFrame(sqlContext, query), overwrite) + // Invalidate the cache. + sqlContext.cacheManager.invalidateCache(logicalRelation) + Seq.empty[Row] } } 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 40fc1f2aa2724..a640ba57e0885 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 @@ -158,6 +158,22 @@ trait CatalystScan extends BaseRelation { } @DeveloperApi +/** + * ::DeveloperApi:: + * A BaseRelation that can be used to insert data into it through the insert method. + * If overwrite in insert method is true, the old data in the relation should be overwritten with + * the new data. If overwrite in insert method is false, the new data should be appended. + * + * InsertableRelation has the following three assumptions. + * 1. It assumes that the data (Rows in the DataFrame) provided to the insert method + * exactly matches the ordinal of fields in the schema of the BaseRelation. + * 2. It assumes that the schema of this relation will not be changed. + * Even if the insert method updates the schema (e.g. a relation of JSON or Parquet data may have a + * schema update after an insert operation), the new schema will not be used. + * 3. It assumes that fields of the data provided in the insert method are nullable. + * 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 { def insert(data: DataFrame, overwrite: Boolean): Unit } 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 new file mode 100644 index 0000000000000..4ed22d363da5b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.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.sources + +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Alias} +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 + +/** + * 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 { + // Wait until children are resolved. + case p: LogicalPlan if !p.childrenResolved => p + + // We are inserting into an InsertableRelation. + case i @ InsertIntoTable( + l @ LogicalRelation(r: InsertableRelation), partition, child, overwrite) => { + // 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) { + sys.error( + s"$l requires that the query in the SELECT clause of the INSERT INTO/OVERWRITE " + + s"statement generates the same number of columns as its schema.") + } + castAndRenameChildOutput(i, l.output, child) + } + } + } + + /** If necessary, cast data types and rename fields to the expected types and names. */ + def castAndRenameChildOutput( + insertInto: InsertIntoTable, + expectedOutput: Seq[Attribute], + child: LogicalPlan) = { + val newChildOutput = expectedOutput.zip(child.output).map { + case (expected, actual) => + val needCast = !DataType.equalsIgnoreNullability(expected.dataType, 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 + (needCast, needRename) match { + case (true, _) => Alias(Cast(actual, expected.dataType), expected.name)() + case (false, true) => Alias(actual, expected.name)() + case (_, _) => actual + } + } + + if (newChildOutput == child.output) { + insertInto + } else { + insertInto.copy(child = Project(newChildOutput, child)) + } + } +} 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 1396c6b7246d1..926ba68828ee8 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 @@ -23,6 +23,7 @@ import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.Dsl._ 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.types._ @@ -923,6 +924,30 @@ class JsonSuite extends QueryTest { sql("select structWithArrayFields.field1[1], structWithArrayFields.field2[3] from complexTable"), Row(5, null) ) + } + test("JSONRelation equality test") { + val relation1 = + JSONRelation("path", 1.0, Some(StructType(StructField("a", IntegerType, true) :: Nil)))(null) + val logicalRelation1 = LogicalRelation(relation1) + val relation2 = + JSONRelation("path", 0.5, Some(StructType(StructField("a", IntegerType, true) :: Nil)))( + org.apache.spark.sql.test.TestSQLContext) + val logicalRelation2 = LogicalRelation(relation2) + val relation3 = + JSONRelation("path", 1.0, Some(StructType(StructField("b", StringType, true) :: Nil)))(null) + val logicalRelation3 = LogicalRelation(relation3) + + assert(relation1 === relation2) + assert(logicalRelation1.sameResult(logicalRelation2), + s"$logicalRelation1 and $logicalRelation2 should be considered having the same result.") + + assert(relation1 !== relation3) + assert(!logicalRelation1.sameResult(logicalRelation3), + s"$logicalRelation1 and $logicalRelation3 should be considered not having the same result.") + + assert(relation2 !== relation3) + assert(!logicalRelation2.sameResult(logicalRelation3), + s"$logicalRelation2 and $logicalRelation3 should be considered not having the same result.") } } 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 9626252e742e5..53f5f7426e9e6 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 @@ -28,7 +28,11 @@ abstract class DataSourceTest extends QueryTest with BeforeAndAfter { implicit val caseInsensisitiveContext = new SQLContext(TestSQLContext.sparkContext) { @transient override protected[sql] lazy val analyzer: Analyzer = - new Analyzer(catalog, functionRegistry, caseSensitive = false) + new Analyzer(catalog, functionRegistry, caseSensitive = false) { + override val extendedRules = + 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/InsertIntoSuite.scala index f91cea6a37060..36e504e759152 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/InsertIntoSuite.scala @@ -63,6 +63,41 @@ class InsertIntoSuite extends DataSourceTest with BeforeAndAfterAll { ) } + test("PreInsert casting and renaming") { + sql( + s""" + |INSERT OVERWRITE TABLE jsonTable SELECT a * 2, a * 4 FROM jt + """.stripMargin) + + checkAnswer( + sql("SELECT a, b FROM jsonTable"), + (1 to 10).map(i => Row(i * 2, s"${i * 4}")) + ) + + sql( + s""" + |INSERT OVERWRITE TABLE jsonTable SELECT a * 4 AS A, a * 6 as c FROM jt + """.stripMargin) + + checkAnswer( + sql("SELECT a, b FROM jsonTable"), + (1 to 10).map(i => Row(i * 4, s"${i * 6}")) + ) + } + + test("SELECT clause generating a different number of columns is not allowed.") { + val message = intercept[RuntimeException] { + sql( + s""" + |INSERT OVERWRITE TABLE jsonTable SELECT a FROM jt + """.stripMargin) + }.getMessage + assert( + message.contains("generates the same number of columns as its schema"), + "SELECT clause generating a different number of columns should not be not allowed." + ) + } + test("INSERT OVERWRITE a JSONRelation multiple times") { sql( s""" @@ -93,4 +128,49 @@ class InsertIntoSuite extends DataSourceTest with BeforeAndAfterAll { """.stripMargin) } } + + test("Caching") { + // Cached Query Execution + cacheTable("jsonTable") + assertCached(sql("SELECT * FROM jsonTable")) + checkAnswer( + sql("SELECT * FROM jsonTable"), + (1 to 10).map(i => Row(i, s"str$i"))) + + assertCached(sql("SELECT a FROM jsonTable")) + checkAnswer( + sql("SELECT a FROM jsonTable"), + (1 to 10).map(Row(_)).toSeq) + + assertCached(sql("SELECT a FROM jsonTable WHERE a < 5")) + checkAnswer( + sql("SELECT a FROM jsonTable WHERE a < 5"), + (1 to 4).map(Row(_)).toSeq) + + assertCached(sql("SELECT a * 2 FROM jsonTable")) + checkAnswer( + sql("SELECT a * 2 FROM jsonTable"), + (1 to 10).map(i => Row(i * 2)).toSeq) + + assertCached(sql("SELECT x.a, y.a FROM jsonTable x JOIN jsonTable y ON x.a = y.a + 1"), 2) + checkAnswer( + sql("SELECT x.a, y.a FROM jsonTable x JOIN jsonTable y ON x.a = y.a + 1"), + (2 to 10).map(i => Row(i, i - 1)).toSeq) + + // Insert overwrite and keep the same schema. + sql( + s""" + |INSERT OVERWRITE TABLE jsonTable SELECT a * 2, b FROM jt + """.stripMargin) + // jsonTable should be recached. + assertCached(sql("SELECT * FROM jsonTable")) + // The cached data is the new data. + checkAnswer( + sql("SELECT a, b FROM jsonTable"), + sql("SELECT a * 2, b FROM jt").collect()) + + // Verify uncaching + uncacheTable("jsonTable") + assertCached(sql("SELECT * FROM jsonTable"), 0) + } } 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 d2371d4a5519e..ad37b7d0e6f59 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 @@ -324,6 +324,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { catalog.PreInsertionCasts :: ExtractPythonUdfs :: ResolveUdtfsAlias :: + sources.PreInsertCastAndRename :: Nil } From b62c35245a6c9bee6a335aa5dcfcca2547e85406 Mon Sep 17 00:00:00 2001 From: OopsOutOfMemory Date: Fri, 6 Feb 2015 12:41:28 -0800 Subject: [PATCH 136/578] [SQL][HiveConsole][DOC] HiveConsole `correct hiveconsole imports` Sorry for that PR #4330 has some mistakes. I correct it.... so it works correctly now. Author: OopsOutOfMemory Closes #4389 from OopsOutOfMemory/doc and squashes the following commits: 843eed9 [OopsOutOfMemory] correct hiveconsole imports --- project/SparkBuild.scala | 1 + sql/README.md | 9 ++++++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f63f9c1982bb5..5e3051c091030 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -276,6 +276,7 @@ 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.hive._ |import org.apache.spark.sql.hive.test.TestHive._ diff --git a/sql/README.md b/sql/README.md index 254ab8eb6376f..a79249965ee67 100644 --- a/sql/README.md +++ b/sql/README.md @@ -29,12 +29,19 @@ From here you can execute queries with HiveQl and manipulate DataFrame by using catalyst$ build/sbt hive/console [info] Starting scala interpreter... +import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.dsl._ +import org.apache.spark.sql.catalyst.errors._ +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.hive._ import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.types._ import org.apache.spark.sql.parquet.ParquetTestData -Welcome to Scala version 2.10.4 (Java HotSpot(TM) 64-Bit Server VM, Java 1.7.0_45). Type in expressions to have them evaluated. Type :help for more information. From bc36356080e3b52aaf61fc1e6b204146ab96d29f Mon Sep 17 00:00:00 2001 From: wangfei Date: Fri, 6 Feb 2015 12:42:23 -0800 Subject: [PATCH 137/578] [SQL][Minor] Remove cache keyword in SqlParser Since cache keyword already defined in `SparkSQLParser` and `SqlParser` of catalyst is a more general parser which should not cover keywords related to underlying compute engine, to remove cache keyword in `SqlParser`. Author: wangfei Closes #4393 from scwf/remove-cache-keyword and squashes the following commits: 10ade16 [wangfei] remove cache keyword in sql parser --- .../src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala | 1 - 1 file changed, 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 a9bd079c7049d..1c588ee2f7176 100755 --- 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 @@ -57,7 +57,6 @@ class SqlParser extends AbstractSparkSQLParser { protected val AVG = Keyword("AVG") protected val BETWEEN = Keyword("BETWEEN") protected val BY = Keyword("BY") - protected val CACHE = Keyword("CACHE") protected val CASE = Keyword("CASE") protected val CAST = Keyword("CAST") protected val COALESCE = Keyword("COALESCE") From 4793c8402a19afe4df51129a7f99e07494a76af2 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 6 Feb 2015 13:08:09 -0800 Subject: [PATCH 138/578] [SPARK-5278][SQL] Introduce UnresolvedGetField and complete the check of ambiguous reference to fields When the `GetField` chain(`a.b.c.d.....`) is interrupted by `GetItem` like `a.b[0].c.d....`, then the check of ambiguous reference to fields is broken. The reason is that: for something like `a.b[0].c.d`, we first parse it to `GetField(GetField(GetItem(Unresolved("a.b"), 0), "c"), "d")`. Then in `LogicalPlan#resolve`, we resolve `"a.b"` and build a `GetField` chain from bottom(the relation). But for the 2 outer `GetFiled`, we have to resolve them in `Analyzer` or do it in `GetField` lazily, check data type of child, search needed field, etc. which is similar to what we have done in `LogicalPlan#resolve`. So in this PR, the fix is just copy the same logic in `LogicalPlan#resolve` to `Analyzer`, which is simple and quick, but I do suggest introduce `UnresolvedGetFiled` like I explained in https://github.com/apache/spark/pull/2405. Author: Wenchen Fan Closes #4068 from cloud-fan/simple and squashes the following commits: a6857b5 [Wenchen Fan] fix import order 8411c40 [Wenchen Fan] use UnresolvedGetField --- .../apache/spark/sql/catalyst/SqlParser.scala | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 34 +++++++++++++------ .../sql/catalyst/analysis/unresolved.scala | 12 +++++++ .../spark/sql/catalyst/dsl/package.scala | 4 +-- .../catalyst/expressions/complexTypes.scala | 24 ++----------- .../sql/catalyst/optimizer/Optimizer.scala | 2 +- .../catalyst/plans/logical/LogicalPlan.scala | 33 ++---------------- .../ExpressionEvaluationSuite.scala | 25 ++++++++++---- .../optimizer/ConstantFoldingSuite.scala | 4 +-- .../scala/org/apache/spark/sql/Column.scala | 3 +- .../org/apache/spark/sql/hive/HiveQl.scala | 2 +- .../hive/execution/HiveResolutionSuite.scala | 27 +++++++++------ 12 files changed, 84 insertions(+), 88 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 1c588ee2f7176..124f083669358 100755 --- 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 @@ -372,7 +372,7 @@ class SqlParser extends AbstractSparkSQLParser { | expression ~ ("[" ~> expression <~ "]") ^^ { case base ~ ordinal => GetItem(base, ordinal) } | (expression <~ ".") ~ ident ^^ - { case base ~ fieldName => GetField(base, fieldName) } + { case base ~ fieldName => UnresolvedGetField(base, fieldName) } | cast | "(" ~> expression <~ ")" | function 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 ae1aee02c64a5..0b59ed1739566 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 @@ -285,7 +285,7 @@ class Analyzer(catalog: Catalog, case q: LogicalPlan => logTrace(s"Attempting to resolve ${q.simpleString}") - q transformExpressions { + q transformExpressionsUp { case u @ UnresolvedAttribute(name) if resolver(name, VirtualColumn.groupingIdName) && q.isInstanceOf[GroupingAnalytics] => // Resolve the virtual column GROUPING__ID for the operator GroupingAnalytics @@ -295,15 +295,8 @@ class Analyzer(catalog: Catalog, val result = q.resolveChildren(name, resolver).getOrElse(u) logDebug(s"Resolving $u to $result") result - - // Resolve field names using the resolver. - case f @ GetField(child, fieldName) if !f.resolved && child.resolved => - child.dataType match { - case StructType(fields) => - val resolvedFieldName = fields.map(_.name).find(resolver(_, fieldName)) - resolvedFieldName.map(n => f.copy(fieldName = n)).getOrElse(f) - case _ => f - } + case UnresolvedGetField(child, fieldName) if child.resolved => + resolveGetField(child, fieldName) } } @@ -312,6 +305,27 @@ class Analyzer(catalog: Catalog, */ 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 = { + 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(", ")}") + } + 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/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index f35921e2a772c..f959a50564011 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 @@ -177,3 +177,15 @@ 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(", ", ", ")") } + +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 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" +} 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 417659eed5957..51a09ac0e1249 100755 --- 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.UnresolvedAttribute +import org.apache.spark.sql.catalyst.analysis.{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} @@ -101,7 +101,7 @@ package object dsl { def isNotNull = IsNotNull(expr) def getItem(ordinal: Expression) = GetItem(expr, ordinal) - def getField(fieldName: String) = GetField(expr, fieldName) + def getField(fieldName: String) = UnresolvedGetField(expr, fieldName) def cast(to: DataType) = Cast(expr, to) 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 1bc34f71441fe..66e2e5c4bafce 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 @@ -73,39 +73,19 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { /** * Returns the value of fields in the Struct `child`. */ -case class GetField(child: Expression, fieldName: String) extends UnaryExpression { +case class GetField(child: Expression, field: StructField, ordinal: Int) extends UnaryExpression { type EvaluatedType = Any def dataType = field.dataType override def nullable = child.nullable || field.nullable override def foldable = child.foldable - protected def structType = child.dataType match { - case s: StructType => s - case otherType => sys.error(s"GetField is not valid on fields of type $otherType") - } - - lazy val field = - structType.fields - .find(_.name == fieldName) - .getOrElse(sys.error(s"No such field $fieldName in ${child.dataType}")) - - lazy val ordinal = structType.fields.indexOf(field) - - override lazy val resolved = childrenResolved && fieldResolved - - /** Returns true only if the fieldName is found in the child struct. */ - private def fieldResolved = child.dataType match { - case StructType(fields) => fields.map(_.name).contains(fieldName) - case _ => false - } - 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.$fieldName" + override def toString = s"$child.${field.name}" } /** 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 376a9f36568a7..8c8f2896eb99b 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 @@ -206,7 +206,7 @@ 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 @ GetField(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/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 65ae066e4b4b5..8d30528328946 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,7 +18,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.Logging -import org.apache.spark.sql.catalyst.analysis.Resolver +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 @@ -160,11 +160,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // One match, but we also need to extract the requested nested field. case Seq((a, nestedFields)) => - val aliased = - Alias( - resolveNesting(nestedFields, a, resolver), - nestedFields.last)() // Preserve the case of the user's field access. - Some(aliased) + Some(Alias(nestedFields.foldLeft(a: Expression)(UnresolvedGetField), nestedFields.last)()) // No matches. case Seq() => @@ -177,31 +173,6 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { this, s"Ambiguous references to $name: ${ambiguousReferences.mkString(",")}") } } - - /** - * Given a list of successive nested field accesses, and a based expression, attempt to resolve - * the actual field lookups on this expression. - */ - private def resolveNesting( - nestedFields: List[String], - expression: Expression, - resolver: Resolver): Expression = { - - (nestedFields, expression.dataType) match { - case (Nil, _) => expression - case (requestedField :: rest, StructType(fields)) => - val actualField = fields.filter(f => resolver(f.name, requestedField)) - if (actualField.length == 0) { - sys.error( - s"No such struct field $requestedField in ${fields.map(_.name).mkString(", ")}") - } else if (actualField.length == 1) { - resolveNesting(rest, GetField(expression, actualField(0).name), resolver) - } else { - sys.error(s"Ambiguous reference to fields ${actualField.mkString(", ")}") - } - case (_, dt) => sys.error(s"Can't access nested field in type $dt") - } - } } /** 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 25d1c105a00a6..7cf6c80194f6c 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 @@ -26,6 +26,7 @@ import org.scalatest.FunSuite import org.scalatest.Matchers._ import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.analysis.UnresolvedGetField import org.apache.spark.sql.types._ @@ -846,23 +847,33 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(GetItem(BoundReference(4, typeArray, true), Literal(null, IntegerType)), null, row) - checkEvaluation(GetField(BoundReference(2, typeS, nullable = true), "a"), "aa", row) - checkEvaluation(GetField(Literal(null, typeS), "a"), null, row) + def quickBuildGetField(expr: Expression, fieldName: String) = { + expr.dataType match { + case StructType(fields) => + val field = fields.find(_.name == fieldName).get + GetField(expr, field, fields.indexOf(field)) + } + } + + 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) val typeS_notNullable = StructType( StructField("a", StringType, nullable = false) :: StructField("b", StringType, nullable = false) :: Nil ) - assert(GetField(BoundReference(2,typeS, nullable = true), "a").nullable === true) - assert(GetField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable === false) + assert(quickBuildGetField(BoundReference(2,typeS, nullable = true), "a").nullable === true) + assert(quickBuildGetField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable === false) - assert(GetField(Literal(null, typeS), "a").nullable === true) - assert(GetField(Literal(null, typeS_notNullable), "a").nullable === true) + assert(quickBuildGetField(Literal(null, typeS), "a").nullable === true) + assert(quickBuildGetField(Literal(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) - checkEvaluation('c.struct(typeS).at(2).getField("a"), "aa", row) + checkEvaluation(quickResolve('c.struct(typeS).at(2).getField("a")), "aa", row) } test("arithmetic") { 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 9fdf3efa02bb6..e22c62505860a 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.EliminateAnalysisOperators +import org.apache.spark.sql.catalyst.analysis.{UnresolvedGetField, EliminateAnalysisOperators} 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 @@ -184,7 +184,7 @@ class ConstantFoldingSuite extends PlanTest { GetItem(Literal(null, ArrayType(IntegerType)), 1) as 'c3, GetItem(Literal(Seq(1), ArrayType(IntegerType)), Literal(null, IntegerType)) as 'c4, - GetField( + UnresolvedGetField( Literal(null, StructType(Seq(StructField("a", IntegerType, true)))), "a") as 'c5, 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 4c2aeadae9492..878b2b0556de7 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 @@ -23,6 +23,7 @@ import scala.language.implicitConversions import org.apache.spark.sql.Dsl.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.analysis.UnresolvedGetField import org.apache.spark.sql.types._ @@ -505,7 +506,7 @@ trait Column extends DataFrame { /** * An expression that gets a field by name in a [[StructField]]. */ - def getField(fieldName: String): Column = exprToColumn(GetField(expr, fieldName)) + def getField(fieldName: String): Column = exprToColumn(UnresolvedGetField(expr, fieldName)) /** * An expression that returns a substring. 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 4b7fa06532963..2a4b88092179f 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 @@ -1038,7 +1038,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C nodeToExpr(qualifier) match { case UnresolvedAttribute(qualifierName) => UnresolvedAttribute(qualifierName + "." + cleanIdentifier(attr)) - case other => GetField(other, attr) + case other => UnresolvedGetField(other, attr) } /* Stars (*) */ 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 178ece447758e..ff8130ae5f6bc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.hive.test.TestHive -import org.apache.spark.sql.hive.test.TestHive.{sparkContext, sql} +import org.apache.spark.sql.hive.test.TestHive.{sparkContext, jsonRDD, sql} import org.apache.spark.sql.hive.test.TestHive.implicits._ case class Nested(a: Int, B: Int) @@ -29,16 +28,24 @@ case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested]) */ class HiveResolutionSuite extends HiveComparisonTest { - case class NestedData(a: Seq[NestedData2], B: NestedData2) - case class NestedData2(a: NestedData3, B: NestedData3) - case class NestedData3(a: Int, B: Int) - test("SPARK-3698: case insensitive test for nested data") { - sparkContext.makeRDD(Seq.empty[NestedData]).registerTempTable("nested") + jsonRDD(sparkContext.makeRDD( + """{"a": [{"a": {"a": 1}}]}""" :: Nil)).registerTempTable("nested") // This should be successfully analyzed sql("SELECT a[0].A.A from nested").queryExecution.analyzed } + test("SPARK-5278: check ambiguous reference to fields") { + jsonRDD(sparkContext.makeRDD( + """{"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] { + sql("SELECT a[0].b from nested").queryExecution.analyzed + } + assert(exception.getMessage.contains("Ambiguous reference to fields")) + } + createQueryTest("table.attr", "SELECT src.key FROM src ORDER BY key LIMIT 1") @@ -68,7 +75,7 @@ class HiveResolutionSuite extends HiveComparisonTest { test("case insensitivity with scala reflection") { // Test resolution with Scala Reflection - TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) + sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) .registerTempTable("caseSensitivityTest") val query = sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") @@ -79,14 +86,14 @@ class HiveResolutionSuite extends HiveComparisonTest { ignore("case insensitivity with scala reflection joins") { // Test resolution with Scala Reflection - TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) + sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) .registerTempTable("caseSensitivityTest") sql("SELECT * FROM casesensitivitytest a JOIN casesensitivitytest b ON a.a = b.a").collect() } test("nested repeated resolution") { - TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) + sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) .registerTempTable("nestedRepeatedTest") assert(sql("SELECT nestedArray[0].a FROM nestedRepeatedTest").collect().head(0) === 1) } From 3d3ecd77411bfdd1c72be51616c46a6caf839be2 Mon Sep 17 00:00:00 2001 From: OopsOutOfMemory Date: Fri, 6 Feb 2015 13:20:10 -0800 Subject: [PATCH 139/578] [SPARK-5586][Spark Shell][SQL] Make `sqlContext` available in spark shell Result is like this ``` 15/02/05 13:41:22 INFO SparkILoop: Created spark context.. Spark context available as sc. 15/02/05 13:41:22 INFO SparkILoop: Created sql context.. SQLContext available as sqlContext. scala> sq sql sqlContext sqlParser sqrt ``` Author: OopsOutOfMemory Closes #4387 from OopsOutOfMemory/sqlContextInShell and squashes the following commits: c7f5203 [OopsOutOfMemory] auto-import sql() function e160697 [OopsOutOfMemory] Merge branch 'sqlContextInShell' of https://github.com/OopsOutOfMemory/spark into sqlContextInShell 37c0a16 [OopsOutOfMemory] auto detect hive support a9c59d9 [OopsOutOfMemory] rename and reduce range of imports 6b9e309 [OopsOutOfMemory] Merge branch 'master' into sqlContextInShell cae652f [OopsOutOfMemory] make sqlContext available in spark shell --- .../org/apache/spark/repl/SparkILoop.scala | 19 +++++++++++++++++++ .../apache/spark/repl/SparkILoopInit.scala | 10 ++++++++++ .../scala/org/apache/spark/repl/Main.scala | 18 ++++++++++++++++++ .../org/apache/spark/repl/SparkILoop.scala | 12 +++++++++++- 4 files changed, 58 insertions(+), 1 deletion(-) 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 72c1a989999b4..b4db3df795177 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 @@ -45,6 +45,7 @@ import scala.reflect.api.{Mirror, TypeCreator, Universe => ApiUniverse} import org.apache.spark.Logging import org.apache.spark.SparkConf import org.apache.spark.SparkContext +import org.apache.spark.sql.SQLContext import org.apache.spark.util.Utils /** The Scala interactive shell. It provides a read-eval-print loop @@ -130,6 +131,7 @@ class SparkILoop( // NOTE: Must be public for visibility @DeveloperApi var sparkContext: SparkContext = _ + var sqlContext: SQLContext = _ override def echoCommandMessage(msg: String) { intp.reporter printMessage msg @@ -1016,6 +1018,23 @@ class SparkILoop( sparkContext } + @DeveloperApi + def createSQLContext(): SQLContext = { + val name = "org.apache.spark.sql.hive.HiveContext" + val loader = Utils.getContextOrSparkClassLoader + try { + sqlContext = loader.loadClass(name).getConstructor(classOf[SparkContext]) + .newInstance(sparkContext).asInstanceOf[SQLContext] + logInfo("Created sql context (with Hive support)..") + } + catch { + case cnf: java.lang.ClassNotFoundException => + sqlContext = new SQLContext(sparkContext) + logInfo("Created sql context..") + } + sqlContext + } + private def getMaster(): String = { val master = this.master match { case Some(m) => m 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 99bd777c04fdb..0cf2de6d399b0 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 @@ -127,7 +127,17 @@ private[repl] trait SparkILoopInit { _sc } """) + command(""" + @transient val sqlContext = { + val _sqlContext = org.apache.spark.repl.Main.interp.createSQLContext() + println("SQL context available as sqlContext.") + _sqlContext + } + """) command("import org.apache.spark.SparkContext._") + command("import sqlContext.implicits._") + command("import sqlContext.sql") + command("import org.apache.spark.sql.Dsl._") } } 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 69e44d4f916e1..dc25692749aad 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 @@ -19,6 +19,7 @@ package org.apache.spark.repl import org.apache.spark.util.Utils import org.apache.spark._ +import org.apache.spark.sql.SQLContext import scala.tools.nsc.Settings import scala.tools.nsc.interpreter.SparkILoop @@ -34,6 +35,7 @@ object Main extends Logging { "-Yrepl-outdir", s"${outputDir.getAbsolutePath}", "-Yrepl-sync"), true) val classServer = new HttpServer(conf, outputDir, new SecurityManager(conf)) var sparkContext: SparkContext = _ + var sqlContext: SQLContext = _ var interp = new SparkILoop // this is a public var because tests reset it. def main(args: Array[String]) { @@ -74,6 +76,22 @@ object Main extends Logging { sparkContext } + def createSQLContext(): SQLContext = { + val name = "org.apache.spark.sql.hive.HiveContext" + val loader = Utils.getContextOrSparkClassLoader + try { + sqlContext = loader.loadClass(name).getConstructor(classOf[SparkContext]) + .newInstance(sparkContext).asInstanceOf[SQLContext] + logInfo("Created sql context (with Hive support)..") + } + catch { + case cnf: java.lang.ClassNotFoundException => + sqlContext = new SQLContext(sparkContext) + logInfo("Created sql context..") + } + sqlContext + } + private def getMaster: String = { val master = { val envMaster = sys.env.get("MASTER") 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 250727305970d..1bd2a6991404b 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 @@ -66,8 +66,18 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter) println("Spark context available as sc.") _sc } - """) + """) + command( """ + @transient val sqlContext = { + val _sqlContext = org.apache.spark.repl.Main.createSQLContext() + println("SQL context available as sqlContext.") + _sqlContext + } + """) command("import org.apache.spark.SparkContext._") + command("import sqlContext.implicits._") + command("import sqlContext.sql") + command("import org.apache.spark.sql.Dsl._") } } From 0f3a36071a44e986d97981032d5b192477b38bbd Mon Sep 17 00:00:00 2001 From: GenTang Date: Fri, 6 Feb 2015 13:27:34 -0800 Subject: [PATCH 140/578] [SPARK-4983] Insert waiting time before tagging EC2 instances The boto API doesn't support tag EC2 instances in the same call that launches them. We add a five-second wait so EC2 has enough time to propagate the information so that the tagging can succeed. Author: GenTang Author: Gen TANG Closes #3986 from GenTang/spark-4983 and squashes the following commits: 13e257d [Gen TANG] modification of comments 47f06755 [GenTang] print the information ab7a931 [GenTang] solve the issus spark-4983 by inserting waiting time 3179737 [GenTang] Revert "handling exceptions about adding tags to ec2" 6a8b53b [GenTang] Revert "the improvement of exception handling" 13e97a6 [GenTang] Revert "typo" 63fd360 [GenTang] typo 692fc2b [GenTang] the improvement of exception handling 6adcf6d [GenTang] handling exceptions about adding tags to ec2 --- ec2/spark_ec2.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 0de4a62e203fd..7371558274d78 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -569,6 +569,9 @@ def launch_cluster(conn, opts, cluster_name): master_nodes = master_res.instances 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..." + time.sleep(5) # Give the instances descriptive names for master in master_nodes: master.add_tag( From 65181b7512c2192311705dfdfad588a80d3a6fde Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 6 Feb 2015 13:52:35 -0800 Subject: [PATCH 141/578] [HOTFIX][MLLIB] fix a compilation error with java 6 Author: Xiangrui Meng Closes #4442 from mengxr/java6-fix and squashes the following commits: 2098500 [Xiangrui Meng] fix a compilation error with java 6 --- .../org/apache/spark/examples/ml/JavaDeveloperApiExample.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 42d4d7d0bef26..4d9dad9f23038 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 (int)get(maxIter); } + int getMaxIter() { return (Integer) get(maxIter); } public MyJavaLogisticRegression() { setMaxIter(100); @@ -124,7 +124,7 @@ public MyJavaLogisticRegression() { // The parameter setter is in this class since it should return type MyJavaLogisticRegression. MyJavaLogisticRegression setMaxIter(int value) { - return (MyJavaLogisticRegression)set(maxIter, value); + return (MyJavaLogisticRegression) set(maxIter, value); } // This method is used by fit(). From b3872e00d155939e40366debda635fc3fb12cc73 Mon Sep 17 00:00:00 2001 From: Vladimir Vladimirov Date: Fri, 6 Feb 2015 13:55:02 -0800 Subject: [PATCH 142/578] SPARK-5633 pyspark saveAsTextFile support for compression codec See https://issues.apache.org/jira/browse/SPARK-5633 for details Author: Vladimir Vladimirov Closes #4403 from smartkiwi/master and squashes the following commits: 94c014e [Vladimir Vladimirov] SPARK-5633 pyspark saveAsTextFile support for compression codec --- python/pyspark/rdd.py | 22 ++++++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 6e029bf7f13fc..bd4f16e058045 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1366,10 +1366,14 @@ def saveAsPickleFile(self, path, batchSize=10): ser = BatchedSerializer(PickleSerializer(), batchSize) self._reserialize(ser)._jrdd.saveAsObjectFile(path) - def saveAsTextFile(self, path): + def saveAsTextFile(self, path, compressionCodecClass=None): """ Save this RDD as a text file, using string representations of elements. + @param path: path to text file + @param compressionCodecClass: (None by default) string i.e. + "org.apache.hadoop.io.compress.GzipCodec" + >>> tempFile = NamedTemporaryFile(delete=True) >>> tempFile.close() >>> sc.parallelize(range(10)).saveAsTextFile(tempFile.name) @@ -1385,6 +1389,16 @@ def saveAsTextFile(self, path): >>> sc.parallelize(['', 'foo', '', 'bar', '']).saveAsTextFile(tempFile2.name) >>> ''.join(sorted(input(glob(tempFile2.name + "/part-0000*")))) '\\n\\n\\nbar\\nfoo\\n' + + Using compressionCodecClass + + >>> tempFile3 = NamedTemporaryFile(delete=True) + >>> tempFile3.close() + >>> 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' """ def func(split, iterator): for x in iterator: @@ -1395,7 +1409,11 @@ def func(split, iterator): yield x keyed = self.mapPartitionsWithIndex(func) keyed._bypass_serializer = True - keyed._jrdd.map(self.ctx._jvm.BytesToString()).saveAsTextFile(path) + if compressionCodecClass: + compressionCodec = self.ctx._jvm.java.lang.Class.forName(compressionCodecClass) + keyed._jrdd.map(self.ctx._jvm.BytesToString()).saveAsTextFile(path, compressionCodec) + else: + keyed._jrdd.map(self.ctx._jvm.BytesToString()).saveAsTextFile(path) # Pair functions From ca66159a4f30d65fa4cd32dbf3ff23978cb7f99b Mon Sep 17 00:00:00 2001 From: Kashish Jain Date: Fri, 6 Feb 2015 13:47:23 -0800 Subject: [PATCH 143/578] SPARK-5613: Catch the ApplicationNotFoundException exception to avoid thread from getting killed on yarn restart. [SPARK-5613] Added a catch block to catch the ApplicationNotFoundException. Without this catch block the thread gets killed on occurrence of this exception. This Exception occurs when yarn restarts and tries to find an application id for a spark job which got interrupted due to yarn getting stopped. See the stacktrace in the bug for more details. Author: Kashish Jain Closes #4392 from kasjain/branch-1.2 and squashes the following commits: 4831000 [Kashish Jain] SPARK-5613: Catch the ApplicationNotFoundException exception to avoid thread from getting killed on yarn restart. --- .../cluster/YarnClientSchedulerBackend.scala | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 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 690f927e938c3..f1b5aafac4066 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 @@ -20,6 +20,7 @@ package org.apache.spark.scheduler.cluster import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} +import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException import org.apache.spark.{SparkException, Logging, SparkContext} import org.apache.spark.deploy.yarn.{Client, ClientArguments} @@ -133,8 +134,14 @@ private[spark] class YarnClientSchedulerBackend( val t = new Thread { override def run() { while (!stopping) { - val report = client.getApplicationReport(appId) - val state = report.getYarnApplicationState() + 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) { From 5687bab8fdfdc5345b8c5b9be8d4595299005fc8 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 6 Feb 2015 14:23:09 -0800 Subject: [PATCH 144/578] [SPARK-5600] [core] Clean up FsHistoryProvider test, fix app sort order. Clean up some test setup code to remove duplicate instantiation of the provider. Also make sure unfinished apps are sorted correctly. Author: Marcelo Vanzin Closes #4370 from vanzin/SPARK-5600 and squashes the following commits: 0d048d5 [Marcelo Vanzin] Cleanup test code a bit. 2585119 [Marcelo Vanzin] Review feedback. 8b97544 [Marcelo Vanzin] Merge branch 'master' into SPARK-5600 be979e9 [Marcelo Vanzin] Merge branch 'master' into SPARK-5600 298371c [Marcelo Vanzin] [SPARK-5600] [core] Clean up FsHistoryProvider test, fix app sort order. --- .../deploy/history/FsHistoryProvider.scala | 15 +++- .../history/FsHistoryProviderSuite.scala | 85 +++++++++---------- 2 files changed, 52 insertions(+), 48 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 92125f2df7d10..868c63d30a202 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 @@ -194,7 +194,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis None } } - .sortBy { info => (-info.endTime, -info.startTime) } + .sortWith(compareAppInfo) lastModifiedTime = newLastModifiedTime @@ -214,7 +214,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis val newIterator = logInfos.iterator.buffered val oldIterator = applications.values.iterator.buffered while (newIterator.hasNext && oldIterator.hasNext) { - if (newIterator.head.endTime > oldIterator.head.endTime) { + if (compareAppInfo(newIterator.head, oldIterator.head)) { addIfAbsent(newIterator.next) } else { addIfAbsent(oldIterator.next) @@ -230,6 +230,17 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis } } + /** + * Comparison function that defines the sort order for the application listing. + * + * @return Whether `i1` should precede `i2`. + */ + private def compareAppInfo( + i1: FsApplicationHistoryInfo, + i2: FsApplicationHistoryInfo): Boolean = { + if (i1.endTime != i2.endTime) i1.endTime >= i2.endTime else i1.startTime >= i2.startTime + } + /** * Replays the events in the specified log file and returns information about the associated * application. 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 1d95432258111..85939eaadccc7 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 @@ -37,13 +37,8 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers private var testDir: File = null - private var provider: FsHistoryProvider = null - before { testDir = Utils.createTempDir() - provider = new FsHistoryProvider(new SparkConf() - .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) - .set("spark.history.fs.updateInterval", "0")) } after { @@ -51,40 +46,41 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers } test("Parse new and old application logs") { - val conf = new SparkConf() - .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) - .set("spark.history.fs.updateInterval", "0") - val provider = new FsHistoryProvider(conf) + val provider = new FsHistoryProvider(createTestConf()) // Write a new-style application log. - val logFile1 = new File(testDir, "new1") - writeFile(logFile1, true, None, - SparkListenerApplicationStart("app1-1", None, 1L, "test"), - SparkListenerApplicationEnd(2L) + val newAppComplete = new File(testDir, "new1") + writeFile(newAppComplete, true, None, + SparkListenerApplicationStart("new-app-complete", None, 1L, "test"), + SparkListenerApplicationEnd(4L) ) // Write an unfinished app, new-style. - val logFile2 = new File(testDir, "new2" + EventLoggingListener.IN_PROGRESS) - writeFile(logFile2, true, None, - SparkListenerApplicationStart("app2-2", None, 1L, "test") + val newAppIncomplete = new File(testDir, "new2" + EventLoggingListener.IN_PROGRESS) + writeFile(newAppIncomplete, true, None, + SparkListenerApplicationStart("new-app-incomplete", None, 1L, "test") ) // Write an old-style application log. - val oldLog = new File(testDir, "old1") - oldLog.mkdir() - createEmptyFile(new File(oldLog, provider.SPARK_VERSION_PREFIX + "1.0")) - writeFile(new File(oldLog, provider.LOG_PREFIX + "1"), false, None, - SparkListenerApplicationStart("app3", None, 2L, "test"), + val oldAppComplete = new File(testDir, "old1") + oldAppComplete.mkdir() + createEmptyFile(new File(oldAppComplete, provider.SPARK_VERSION_PREFIX + "1.0")) + writeFile(new File(oldAppComplete, provider.LOG_PREFIX + "1"), false, None, + SparkListenerApplicationStart("old-app-complete", None, 2L, "test"), SparkListenerApplicationEnd(3L) ) - createEmptyFile(new File(oldLog, provider.APPLICATION_COMPLETE)) + createEmptyFile(new File(oldAppComplete, provider.APPLICATION_COMPLETE)) + + // Check for logs so that we force the older unfinished app to be loaded, to make + // sure unfinished apps are also sorted correctly. + provider.checkForLogs() // Write an unfinished app, old-style. - val oldLog2 = new File(testDir, "old2") - oldLog2.mkdir() - createEmptyFile(new File(oldLog2, provider.SPARK_VERSION_PREFIX + "1.0")) - writeFile(new File(oldLog2, provider.LOG_PREFIX + "1"), false, None, - SparkListenerApplicationStart("app4", None, 2L, "test") + val oldAppIncomplete = new File(testDir, "old2") + oldAppIncomplete.mkdir() + createEmptyFile(new File(oldAppIncomplete, provider.SPARK_VERSION_PREFIX + "1.0")) + writeFile(new File(oldAppIncomplete, provider.LOG_PREFIX + "1"), false, None, + SparkListenerApplicationStart("old-app-incomplete", None, 2L, "test") ) // Force a reload of data from the log directory, and check that both logs are loaded. @@ -96,14 +92,14 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers list.size should be (4) list.count(e => e.completed) should be (2) - list(0) should be (ApplicationHistoryInfo(oldLog.getName(), "app3", 2L, 3L, - oldLog.lastModified(), "test", true)) - list(1) should be (ApplicationHistoryInfo(logFile1.getName(), "app1-1", 1L, 2L, - logFile1.lastModified(), "test", true)) - list(2) should be (ApplicationHistoryInfo(oldLog2.getName(), "app4", 2L, -1L, - oldLog2.lastModified(), "test", false)) - list(3) should be (ApplicationHistoryInfo(logFile2.getName(), "app2-2", 1L, -1L, - logFile2.lastModified(), "test", false)) + list(0) should be (ApplicationHistoryInfo(newAppComplete.getName(), "new-app-complete", 1L, 4L, + newAppComplete.lastModified(), "test", true)) + list(1) 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, + -1L, oldAppIncomplete.lastModified(), "test", false)) + list(3) should be (ApplicationHistoryInfo(newAppIncomplete.getName(), "new-app-incomplete", 1L, + -1L, newAppIncomplete.lastModified(), "test", false)) // Make sure the UI can be rendered. list.foreach { case info => @@ -113,6 +109,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers } test("Parse legacy logs with compression codec set") { + val provider = new FsHistoryProvider(createTestConf()) val testCodecs = List((classOf[LZFCompressionCodec].getName(), true), (classOf[SnappyCompressionCodec].getName(), true), ("invalid.codec", false)) @@ -156,10 +153,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers ) logFile2.setReadable(false, false) - val conf = new SparkConf() - .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) - .set("spark.history.fs.updateInterval", "0") - val provider = new FsHistoryProvider(conf) + val provider = new FsHistoryProvider(createTestConf()) provider.checkForLogs() val list = provider.getListing().toSeq @@ -168,10 +162,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers } test("history file is renamed from inprogress to completed") { - val conf = new SparkConf() - .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) - .set("spark.testing", "true") - val provider = new FsHistoryProvider(conf) + val provider = new FsHistoryProvider(createTestConf()) val logFile1 = new File(testDir, "app1" + EventLoggingListener.IN_PROGRESS) writeFile(logFile1, true, None, @@ -191,9 +182,7 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers } test("SPARK-5582: empty log directory") { - val conf = new SparkConf() - .set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) - val provider = new FsHistoryProvider(conf) + val provider = new FsHistoryProvider(createTestConf()) val logFile1 = new File(testDir, "app1" + EventLoggingListener.IN_PROGRESS) writeFile(logFile1, true, None, @@ -229,4 +218,8 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers new FileOutputStream(file).close() } + private def createTestConf(): SparkConf = { + new SparkConf().set("spark.history.fs.logDirectory", testDir.getAbsolutePath()) + } + } From 57961567ef104efb3174e67d762c5d9d6263b800 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 6 Feb 2015 14:27:06 -0800 Subject: [PATCH 145/578] [HOTFIX] Fix the maven build after adding sqlContext to spark-shell Follow up to #4387 to fix the build break. Author: Michael Armbrust Closes #4443 from marmbrus/fixMaven and squashes the following commits: 1eeba7d [Michael Armbrust] try again 7f5fb15 [Michael Armbrust] [HOTFIX] Fix the maven build after adding sqlContext to spark-shell --- repl/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/repl/pom.xml b/repl/pom.xml index bd39b90fd8714..3d4adf8fd5b03 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -66,7 +66,6 @@ org.apache.spark spark-sql_${scala.binary.version} ${project.version} - test org.scala-lang From dcd1e42d6b6ac08d2c0736bf61a15f515a1f222b Mon Sep 17 00:00:00 2001 From: Kostas Sakellis Date: Fri, 6 Feb 2015 14:31:20 -0800 Subject: [PATCH 146/578] [SPARK-4874] [CORE] Collect record count metrics Collects record counts for both Input/Output and Shuffle Metrics. For the input/output metrics, it just appends the counter every time the iterators get accessed. For shuffle on the write side, we count the metrics post aggregation (after a map side combine) and on the read side we count the metrics pre aggregation. This allows both the bytes read/written metrics and the records read/written to line up. For backwards compatibility, if we deserialize an older event that doesn't have record metrics, we set the metric to -1. Author: Kostas Sakellis Closes #4067 from ksakellis/kostas-spark-4874 and squashes the following commits: bd919be [Kostas Sakellis] Changed 'Records Read' in shuffleReadMetrics json output to 'Total Records Read' dad4d57 [Kostas Sakellis] Add a comment and check to BlockObjectWriter so that it cannot be reopend. 6f236a1 [Kostas Sakellis] Renamed _recordsWritten in ShuffleWriteMetrics to be more consistent 70620a0 [Kostas Sakellis] CR Feedback 17faa3a [Kostas Sakellis] Removed AtomicLong in favour of using Long b6f9923 [Kostas Sakellis] Merge AfterNextInterceptingIterator with InterruptableIterator to save a function call 46c8186 [Kostas Sakellis] Combined Bytes and # records into one column 57551c1 [Kostas Sakellis] Conforms to SPARK-3288 6cdb44e [Kostas Sakellis] Removed the generic InterceptingIterator and repalced it with specific implementation 1aa273c [Kostas Sakellis] CR Feedback 1bb78b1 [Kostas Sakellis] [SPARK-4874] [CORE] Collect record count metrics --- .../scala/org/apache/spark/CacheManager.scala | 11 +- .../apache/spark/executor/TaskMetrics.scala | 54 +++-- .../org/apache/spark/rdd/HadoopRDD.scala | 6 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 6 +- .../apache/spark/rdd/PairRDDFunctions.scala | 12 +- .../hash/BlockStoreShuffleFetcher.scala | 8 +- .../apache/spark/storage/BlockManager.scala | 2 +- .../spark/storage/BlockObjectWriter.scala | 24 ++- .../scala/org/apache/spark/ui/ToolTips.scala | 9 +- .../apache/spark/ui/exec/ExecutorsTab.scala | 6 + .../apache/spark/ui/jobs/ExecutorTable.scala | 86 ++++++-- .../spark/ui/jobs/JobProgressListener.scala | 24 +++ .../org/apache/spark/ui/jobs/StagePage.scala | 148 +++++++++----- .../org/apache/spark/ui/jobs/UIData.scala | 14 ++ .../org/apache/spark/util/JsonProtocol.scala | 19 +- .../util/collection/ExternalSorter.scala | 1 + .../metrics/InputOutputMetricsSuite.scala | 186 ++++++++++++++++-- .../storage/BlockObjectWriterSuite.scala | 21 ++ .../ui/jobs/JobProgressListenerSuite.scala | 2 +- .../apache/spark/util/JsonProtocolSuite.scala | 55 +++++- 20 files changed, 548 insertions(+), 146 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index a0c0372b7f0ef..a96d754744a05 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -47,10 +47,15 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { val inputMetrics = blockResult.inputMetrics val existingMetrics = context.taskMetrics .getInputMetricsForReadMethod(inputMetrics.readMethod) - existingMetrics.addBytesRead(inputMetrics.bytesRead) - - new InterruptibleIterator(context, blockResult.data.asInstanceOf[Iterator[T]]) + existingMetrics.incBytesRead(inputMetrics.bytesRead) + val iter = blockResult.data.asInstanceOf[Iterator[T]] + new InterruptibleIterator[T](context, iter) { + override def next(): T = { + existingMetrics.incRecordsRead(1) + delegate.next() + } + } case None => // Acquire a lock for loading this partition // If another thread already holds the lock, wait for it to finish return its results 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 97912c68c5982..d05659193b334 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -194,18 +194,19 @@ class TaskMetrics extends Serializable { /** * Aggregates shuffle read metrics for all registered dependencies into shuffleReadMetrics. */ - private[spark] def updateShuffleReadMetrics() = synchronized { + 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) } _shuffleReadMetrics = Some(merged) } - private[spark] def updateInputMetrics() = synchronized { + private[spark] def updateInputMetrics(): Unit = synchronized { inputMetrics.foreach(_.updateBytesRead()) } } @@ -242,27 +243,31 @@ object DataWriteMethod extends Enumeration with Serializable { @DeveloperApi case class InputMetrics(readMethod: DataReadMethod.Value) { - private val _bytesRead: AtomicLong = new AtomicLong() + /** + * This is volatile so that it is visible to the updater thread. + */ + @volatile @transient var bytesReadCallback: Option[() => Long] = None /** * Total bytes read. */ - def bytesRead: Long = _bytesRead.get() - @volatile @transient var bytesReadCallback: Option[() => Long] = None + private var _bytesRead: Long = _ + def bytesRead: Long = _bytesRead + def incBytesRead(bytes: Long) = _bytesRead += bytes /** - * Adds additional bytes read for this read method. + * Total records read. */ - def addBytesRead(bytes: Long) = { - _bytesRead.addAndGet(bytes) - } + private var _recordsRead: Long = _ + def recordsRead: Long = _recordsRead + def incRecordsRead(records: Long) = _recordsRead += records /** * Invoke the bytesReadCallback and mutate bytesRead. */ def updateBytesRead() { bytesReadCallback.foreach { c => - _bytesRead.set(c()) + _bytesRead = c() } } @@ -287,6 +292,13 @@ case class OutputMetrics(writeMethod: DataWriteMethod.Value) { private var _bytesWritten: Long = _ def bytesWritten = _bytesWritten private[spark] def setBytesWritten(value : Long) = _bytesWritten = value + + /** + * Total records written + */ + private var _recordsWritten: Long = 0L + def recordsWritten = _recordsWritten + private[spark] def setRecordsWritten(value: Long) = _recordsWritten = value } /** @@ -301,7 +313,7 @@ class ShuffleReadMetrics extends Serializable { private var _remoteBlocksFetched: Int = _ def remoteBlocksFetched = _remoteBlocksFetched private[spark] def incRemoteBlocksFetched(value: Int) = _remoteBlocksFetched += value - private[spark] def defRemoteBlocksFetched(value: Int) = _remoteBlocksFetched -= value + private[spark] def decRemoteBlocksFetched(value: Int) = _remoteBlocksFetched -= value /** * Number of local blocks fetched in this shuffle by this task @@ -309,8 +321,7 @@ class ShuffleReadMetrics extends Serializable { private var _localBlocksFetched: Int = _ def localBlocksFetched = _localBlocksFetched private[spark] def incLocalBlocksFetched(value: Int) = _localBlocksFetched += value - private[spark] def defLocalBlocksFetched(value: Int) = _localBlocksFetched -= value - + private[spark] def decLocalBlocksFetched(value: Int) = _localBlocksFetched -= value /** * Time the task spent waiting for remote shuffle blocks. This only includes the time @@ -334,6 +345,14 @@ class ShuffleReadMetrics extends Serializable { * Number of blocks fetched in this shuffle by this task (remote or local) */ def totalBlocksFetched = _remoteBlocksFetched + _localBlocksFetched + + /** + * Total number of records read from the shuffle by this task + */ + private var _recordsRead: Long = _ + def recordsRead = _recordsRead + private[spark] def incRecordsRead(value: Long) = _recordsRead += value + private[spark] def decRecordsRead(value: Long) = _recordsRead -= value } /** @@ -358,5 +377,12 @@ class ShuffleWriteMetrics extends Serializable { private[spark] def incShuffleWriteTime(value: Long) = _shuffleWriteTime += value private[spark] def decShuffleWriteTime(value: Long) = _shuffleWriteTime -= value - + /** + * Total number of records written to the shuffle by this task + */ + @volatile private var _shuffleRecordsWritten: Long = _ + def shuffleRecordsWritten = _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/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 89adddcf0ac36..486e86ce1bb19 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -247,7 +247,9 @@ class HadoopRDD[K, V]( case eof: EOFException => finished = true } - + if (!finished) { + inputMetrics.incRecordsRead(1) + } (key, value) } @@ -261,7 +263,7 @@ class HadoopRDD[K, V]( // If we can't get the bytes read from the FS stats, fall back to the split size, // which may be inaccurate. try { - inputMetrics.addBytesRead(split.inputSplit.value.getLength) + inputMetrics.incBytesRead(split.inputSplit.value.getLength) } catch { case e: java.io.IOException => logWarning("Unable to get input size to set InputMetrics for task", e) 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 44b9ffd2a53fd..7fb94840df99c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -151,7 +151,9 @@ class NewHadoopRDD[K, V]( throw new java.util.NoSuchElementException("End of stream") } havePair = false - + if (!finished) { + inputMetrics.incRecordsRead(1) + } (reader.getCurrentKey, reader.getCurrentValue) } @@ -165,7 +167,7 @@ class NewHadoopRDD[K, V]( // If we can't get the bytes read from the FS stats, fall back to the split size, // which may be inaccurate. try { - inputMetrics.addBytesRead(split.serializableHadoopSplit.value.getLength) + inputMetrics.incBytesRead(split.serializableHadoopSplit.value.getLength) } catch { case e: java.io.IOException => logWarning("Unable to get input size to set InputMetrics for task", e) 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 49b88a90ab5af..955b42c3baaa1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -34,7 +34,7 @@ import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob, OutputFormat => NewOutputFormat, -RecordWriter => NewRecordWriter} + RecordWriter => NewRecordWriter} import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner @@ -993,8 +993,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val (outputMetrics, bytesWrittenCallback) = initHadoopOutputMetrics(context) val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K,V]] + var recordsWritten = 0L try { - var recordsWritten = 0L while (iter.hasNext) { val pair = iter.next() writer.write(pair._1, pair._2) @@ -1008,6 +1008,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) } committer.commitTask(hadoopContext) bytesWrittenCallback.foreach { fn => outputMetrics.setBytesWritten(fn()) } + outputMetrics.setRecordsWritten(recordsWritten) 1 } : Int @@ -1065,8 +1066,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) writer.setup(context.stageId, context.partitionId, taskAttemptId) writer.open() + var recordsWritten = 0L try { - var recordsWritten = 0L while (iter.hasNext) { val record = iter.next() writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef]) @@ -1080,6 +1081,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) } writer.commit() bytesWrittenCallback.foreach { fn => outputMetrics.setBytesWritten(fn()) } + outputMetrics.setRecordsWritten(recordsWritten) } self.context.runJob(self, writeToFile) @@ -1097,9 +1099,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) private def maybeUpdateOutputMetrics(bytesWrittenCallback: Option[() => Long], outputMetrics: OutputMetrics, recordsWritten: Long): Unit = { - if (recordsWritten % PairRDDFunctions.RECORDS_BETWEEN_BYTES_WRITTEN_METRIC_UPDATES == 0 - && bytesWrittenCallback.isDefined) { + if (recordsWritten % PairRDDFunctions.RECORDS_BETWEEN_BYTES_WRITTEN_METRIC_UPDATES == 0) { bytesWrittenCallback.foreach { fn => outputMetrics.setBytesWritten(fn()) } + outputMetrics.setRecordsWritten(recordsWritten) } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index e3e7434df45b0..7a2c5ae32d98b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -86,6 +86,12 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { context.taskMetrics.updateShuffleReadMetrics() }) - new InterruptibleIterator[T](context, completionIter) + new InterruptibleIterator[T](context, completionIter) { + val readMetrics = context.taskMetrics.createShuffleReadMetricsForDependency() + override def next(): T = { + readMetrics.incRecordsRead(1) + delegate.next() + } + } } } 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 8bc5a1cd18b64..86dbd89f0ffb8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -53,7 +53,7 @@ private[spark] class BlockResult( readMethod: DataReadMethod.Value, bytes: Long) { val inputMetrics = new InputMetrics(readMethod) - inputMetrics.addBytesRead(bytes) + inputMetrics.incBytesRead(bytes) } /** 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 3198d766fca37..81164178b9e8e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -29,7 +29,8 @@ import org.apache.spark.executor.ShuffleWriteMetrics * appending data to an existing block, and can guarantee atomicity in the case of faults * as it allows the caller to revert partial writes. * - * This interface does not support concurrent writes. + * This interface does not support concurrent writes. Also, once the writer has + * been opened, it cannot be reopened again. */ private[spark] abstract class BlockObjectWriter(val blockId: BlockId) { @@ -95,6 +96,7 @@ private[spark] class DiskBlockObjectWriter( private var ts: TimeTrackingOutputStream = null private var objOut: SerializationStream = null private var initialized = false + private var hasBeenClosed = false /** * Cursors used to represent positions in the file. @@ -115,11 +117,16 @@ private[spark] class DiskBlockObjectWriter( private var finalPosition: Long = -1 private var reportedPosition = initialPosition - /** Calling channel.position() to update the write metrics can be a little bit expensive, so we - * only call it every N writes */ - private var writesSinceMetricsUpdate = 0 + /** + * Keep track of number of records written and also use this to periodically + * output bytes written since the latter is expensive to do for each record. + */ + private var numRecordsWritten = 0 override def open(): BlockObjectWriter = { + if (hasBeenClosed) { + throw new IllegalStateException("Writer already closed. Cannot be reopened.") + } fos = new FileOutputStream(file, true) ts = new TimeTrackingOutputStream(fos) channel = fos.getChannel() @@ -145,6 +152,7 @@ private[spark] class DiskBlockObjectWriter( ts = null objOut = null initialized = false + hasBeenClosed = true } } @@ -168,6 +176,7 @@ private[spark] class DiskBlockObjectWriter( override def revertPartialWritesAndClose() { try { writeMetrics.decShuffleBytesWritten(reportedPosition - initialPosition) + writeMetrics.decShuffleRecordsWritten(numRecordsWritten) if (initialized) { objOut.flush() @@ -193,12 +202,11 @@ private[spark] class DiskBlockObjectWriter( } objOut.writeObject(value) + numRecordsWritten += 1 + writeMetrics.incShuffleRecordsWritten(1) - if (writesSinceMetricsUpdate == 32) { - writesSinceMetricsUpdate = 0 + if (numRecordsWritten % 32 == 0) { updateBytesWritten() - } else { - writesSinceMetricsUpdate += 1 } } 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 4307029d44fbb..3a15e603b1969 100644 --- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala +++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala @@ -29,14 +29,15 @@ private[spark] object ToolTips { val SHUFFLE_READ_BLOCKED_TIME = "Time that the task spent blocked waiting for shuffle data to be read from remote machines." - val INPUT = "Bytes read from Hadoop or from Spark storage." + val INPUT = "Bytes and records read from Hadoop or from Spark storage." - val OUTPUT = "Bytes written to Hadoop." + val OUTPUT = "Bytes and records written to Hadoop." - val SHUFFLE_WRITE = "Bytes written to disk in order to be read by a shuffle in a future stage." + val SHUFFLE_WRITE = + "Bytes and records written to disk in order to be read by a shuffle in a future stage." val SHUFFLE_READ = - """Bytes read from remote executors. Typically less than shuffle write bytes + """Bytes and records read from remote executors. Typically less than shuffle write bytes because this does not include shuffle data read locally.""" val GETTING_RESULT_TIME = 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 a38cb75fdd8c6..3afd7ef07d7c9 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 @@ -48,7 +48,9 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) extends Sp val executorToTasksFailed = HashMap[String, Int]() val executorToDuration = HashMap[String, Long]() val executorToInputBytes = HashMap[String, Long]() + val executorToInputRecords = HashMap[String, Long]() val executorToOutputBytes = HashMap[String, Long]() + val executorToOutputRecords = HashMap[String, Long]() val executorToShuffleRead = HashMap[String, Long]() val executorToShuffleWrite = HashMap[String, Long]() val executorToLogUrls = HashMap[String, Map[String, String]]() @@ -84,10 +86,14 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) extends Sp metrics.inputMetrics.foreach { inputMetrics => executorToInputBytes(eid) = executorToInputBytes.getOrElse(eid, 0L) + inputMetrics.bytesRead + executorToInputRecords(eid) = + executorToInputRecords.getOrElse(eid, 0L) + inputMetrics.recordsRead } metrics.outputMetrics.foreach { outputMetrics => executorToOutputBytes(eid) = executorToOutputBytes.getOrElse(eid, 0L) + outputMetrics.bytesWritten + executorToOutputRecords(eid) = + executorToOutputRecords.getOrElse(eid, 0L) + outputMetrics.recordsWritten } metrics.shuffleReadMetrics.foreach { shuffleRead => executorToShuffleRead(eid) = diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 9836d11a6d85f..1f8536d1b7195 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -36,6 +36,20 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage /** Special table which merges two header cells. */ private def executorTable[T](): Seq[Node] = { + val stageData = listener.stageIdToData.get((stageId, stageAttemptId)) + var hasInput = false + var hasOutput = false + var hasShuffleWrite = false + var hasShuffleRead = false + var hasBytesSpilled = false + stageData.foreach(data => { + hasInput = data.hasInput + hasOutput = data.hasOutput + hasShuffleRead = data.hasShuffleRead + hasShuffleWrite = data.hasShuffleWrite + hasBytesSpilled = data.hasBytesSpilled + }) + @@ -44,12 +58,32 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage - - - - - - + {if (hasInput) { + + }} + {if (hasOutput) { + + }} + {if (hasShuffleRead) { + + }} + {if (hasShuffleWrite) { + + }} + {if (hasBytesSpilled) { + + + }} {createExecutorTable()} @@ -76,18 +110,34 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage - - - - - - + {if (stageData.hasInput) { + + }} + {if (stageData.hasOutput) { + + }} + {if (stageData.hasShuffleRead) { + + }} + {if (stageData.hasShuffleWrite) { + + }} + {if (stageData.hasBytesSpilled) { + + + }} } case None => 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 4d200eeda86b9..f463f8d7c7215 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 @@ -394,24 +394,48 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { stageData.shuffleWriteBytes += shuffleWriteDelta execSummary.shuffleWrite += shuffleWriteDelta + val shuffleWriteRecordsDelta = + (taskMetrics.shuffleWriteMetrics.map(_.shuffleRecordsWritten).getOrElse(0L) + - oldMetrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleRecordsWritten).getOrElse(0L)) + stageData.shuffleWriteRecords += shuffleWriteRecordsDelta + execSummary.shuffleWriteRecords += shuffleWriteRecordsDelta + val shuffleReadDelta = (taskMetrics.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L) - oldMetrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead).getOrElse(0L)) stageData.shuffleReadBytes += shuffleReadDelta execSummary.shuffleRead += shuffleReadDelta + val shuffleReadRecordsDelta = + (taskMetrics.shuffleReadMetrics.map(_.recordsRead).getOrElse(0L) + - oldMetrics.flatMap(_.shuffleReadMetrics).map(_.recordsRead).getOrElse(0L)) + stageData.shuffleReadRecords += shuffleReadRecordsDelta + execSummary.shuffleReadRecords += shuffleReadRecordsDelta + val inputBytesDelta = (taskMetrics.inputMetrics.map(_.bytesRead).getOrElse(0L) - oldMetrics.flatMap(_.inputMetrics).map(_.bytesRead).getOrElse(0L)) stageData.inputBytes += inputBytesDelta execSummary.inputBytes += inputBytesDelta + val inputRecordsDelta = + (taskMetrics.inputMetrics.map(_.recordsRead).getOrElse(0L) + - oldMetrics.flatMap(_.inputMetrics).map(_.recordsRead).getOrElse(0L)) + stageData.inputRecords += inputRecordsDelta + execSummary.inputRecords += inputRecordsDelta + val outputBytesDelta = (taskMetrics.outputMetrics.map(_.bytesWritten).getOrElse(0L) - oldMetrics.flatMap(_.outputMetrics).map(_.bytesWritten).getOrElse(0L)) stageData.outputBytes += outputBytesDelta execSummary.outputBytes += outputBytesDelta + val outputRecordsDelta = + (taskMetrics.outputMetrics.map(_.recordsWritten).getOrElse(0L) + - oldMetrics.flatMap(_.outputMetrics).map(_.recordsWritten).getOrElse(0L)) + stageData.outputRecords += outputRecordsDelta + execSummary.outputRecords += outputRecordsDelta + val diskSpillDelta = taskMetrics.diskBytesSpilled - oldMetrics.map(_.diskBytesSpilled).getOrElse(0L) stageData.diskBytesSpilled += diskSpillDelta 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 d8be1b20b3acd..02a3cc3e43c25 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 @@ -56,11 +56,6 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val numCompleted = tasks.count(_.taskInfo.finished) val accumulables = listener.stageIdToData((stageId, stageAttemptId)).accumulables val hasAccumulators = accumulables.size > 0 - val hasInput = stageData.inputBytes > 0 - val hasOutput = stageData.outputBytes > 0 - val hasShuffleRead = stageData.shuffleReadBytes > 0 - val hasShuffleWrite = stageData.shuffleWriteBytes > 0 - val hasBytesSpilled = stageData.memoryBytesSpilled > 0 && stageData.diskBytesSpilled > 0 val summary =
@@ -69,31 +64,33 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { Total task time across all tasks: {UIUtils.formatDuration(stageData.executorRunTime)} - {if (hasInput) { + {if (stageData.hasInput) {
  • - Input: - {Utils.bytesToString(stageData.inputBytes)} + Input Size / Records: + {s"${Utils.bytesToString(stageData.inputBytes)} / ${stageData.inputRecords}"}
  • }} - {if (hasOutput) { + {if (stageData.hasOutput) {
  • Output: - {Utils.bytesToString(stageData.outputBytes)} + {s"${Utils.bytesToString(stageData.outputBytes)} / ${stageData.outputRecords}"}
  • }} - {if (hasShuffleRead) { + {if (stageData.hasShuffleRead) {
  • Shuffle read: - {Utils.bytesToString(stageData.shuffleReadBytes)} + {s"${Utils.bytesToString(stageData.shuffleReadBytes)} / " + + s"${stageData.shuffleReadRecords}"}
  • }} - {if (hasShuffleWrite) { + {if (stageData.hasShuffleWrite) {
  • Shuffle write: - {Utils.bytesToString(stageData.shuffleWriteBytes)} + {s"${Utils.bytesToString(stageData.shuffleWriteBytes)} / " + + s"${stageData.shuffleWriteRecords}"}
  • }} - {if (hasBytesSpilled) { + {if (stageData.hasBytesSpilled) {
  • Shuffle spill (memory): {Utils.bytesToString(stageData.memoryBytesSpilled)} @@ -132,7 +129,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { Task Deserialization Time
  • - {if (hasShuffleRead) { + {if (stageData.hasShuffleRead) {
  • @@ -174,25 +171,32 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { ("Result Serialization Time", TaskDetailsClassNames.RESULT_SERIALIZATION_TIME), ("Getting Result Time", TaskDetailsClassNames.GETTING_RESULT_TIME)) ++ {if (hasAccumulators) Seq(("Accumulators", "")) else Nil} ++ - {if (hasInput) Seq(("Input", "")) else Nil} ++ - {if (hasOutput) Seq(("Output", "")) else Nil} ++ - {if (hasShuffleRead) { + {if (stageData.hasInput) Seq(("Input Size / Records", "")) else Nil} ++ + {if (stageData.hasOutput) Seq(("Output Size / Records", "")) else Nil} ++ + {if (stageData.hasShuffleRead) { Seq(("Shuffle Read Blocked Time", TaskDetailsClassNames.SHUFFLE_READ_BLOCKED_TIME), - ("Shuffle Read", "")) + ("Shuffle Read Size / Records", "")) + } else { + Nil + }} ++ + {if (stageData.hasShuffleWrite) { + Seq(("Write Time", ""), ("Shuffle Write Size / Records", "")) + } else { + Nil + }} ++ + {if (stageData.hasBytesSpilled) { + Seq(("Shuffle Spill (Memory)", ""), ("Shuffle Spill (Disk)", "")) } else { Nil }} ++ - {if (hasShuffleWrite) Seq(("Write Time", ""), ("Shuffle Write", "")) else Nil} ++ - {if (hasBytesSpilled) Seq(("Shuffle Spill (Memory)", ""), ("Shuffle Spill (Disk)", "")) - else Nil} ++ Seq(("Errors", "")) val unzipped = taskHeadersAndCssClasses.unzip val taskTable = UIUtils.listingTable( unzipped._1, - taskRow(hasAccumulators, hasInput, hasOutput, hasShuffleRead, hasShuffleWrite, - hasBytesSpilled), + taskRow(hasAccumulators, stageData.hasInput, stageData.hasOutput, + stageData.hasShuffleRead, stageData.hasShuffleWrite, stageData.hasBytesSpilled), tasks, headerClasses = unzipped._2) // Excludes tasks which failed and have incomplete metrics @@ -203,8 +207,11 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { None } else { + def getDistributionQuantiles(data: Seq[Double]): IndexedSeq[Double] = + Distribution(data).get.getQuantiles() + def getFormattedTimeQuantiles(times: Seq[Double]): Seq[Node] = { - Distribution(times).get.getQuantiles().map { millis => + getDistributionQuantiles(times).map { millis =>
  • } } @@ -273,17 +280,36 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { getFormattedTimeQuantiles(schedulerDelays) def getFormattedSizeQuantiles(data: Seq[Double]) = - Distribution(data).get.getQuantiles().map(d => ) + getDistributionQuantiles(data).map(d => ) + + def getFormattedSizeQuantilesWithRecords(data: Seq[Double], records: Seq[Double]) = { + val recordDist = getDistributionQuantiles(records).iterator + getDistributionQuantiles(data).map(d => + + ) + } val inputSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.inputMetrics.map(_.bytesRead).getOrElse(0L).toDouble } - val inputQuantiles = +: getFormattedSizeQuantiles(inputSizes) + + val inputRecords = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.inputMetrics.map(_.recordsRead).getOrElse(0L).toDouble + } + + val inputQuantiles = +: + getFormattedSizeQuantilesWithRecords(inputSizes, inputRecords) val outputSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.outputMetrics.map(_.bytesWritten).getOrElse(0L).toDouble } - val outputQuantiles = +: getFormattedSizeQuantiles(outputSizes) + + val outputRecords = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.outputMetrics.map(_.recordsWritten).getOrElse(0L).toDouble + } + + val outputQuantiles = +: + getFormattedSizeQuantilesWithRecords(outputSizes, outputRecords) val shuffleReadBlockedTimes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.shuffleReadMetrics.map(_.fetchWaitTime).getOrElse(0L).toDouble @@ -294,14 +320,24 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val shuffleReadSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble } - val shuffleReadQuantiles = +: - getFormattedSizeQuantiles(shuffleReadSizes) + + val shuffleReadRecords = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.shuffleReadMetrics.map(_.recordsRead).getOrElse(0L).toDouble + } + + val shuffleReadQuantiles = +: + getFormattedSizeQuantilesWithRecords(shuffleReadSizes, shuffleReadRecords) val shuffleWriteSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble } - val shuffleWriteQuantiles = +: - getFormattedSizeQuantiles(shuffleWriteSizes) + + val shuffleWriteRecords = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.shuffleWriteMetrics.map(_.shuffleRecordsWritten).getOrElse(0L).toDouble + } + + val shuffleWriteQuantiles = +: + getFormattedSizeQuantilesWithRecords(shuffleWriteSizes, shuffleWriteRecords) val memoryBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.memoryBytesSpilled.toDouble @@ -326,9 +362,9 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { {serializationQuantiles} , {gettingResultQuantiles}, - if (hasInput) {inputQuantiles} else Nil, - if (hasOutput) {outputQuantiles} else Nil, - if (hasShuffleRead) { + if (stageData.hasInput) {inputQuantiles} else Nil, + if (stageData.hasOutput) {outputQuantiles} else Nil, + if (stageData.hasShuffleRead) { {shuffleReadBlockedQuantiles} @@ -336,9 +372,9 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { } else { Nil }, - if (hasShuffleWrite) {shuffleWriteQuantiles} else Nil, - if (hasBytesSpilled) {memoryBytesSpilledQuantiles} else Nil, - if (hasBytesSpilled) {diskBytesSpilledQuantiles} else Nil) + if (stageData.hasShuffleWrite) {shuffleWriteQuantiles} else Nil, + if (stageData.hasBytesSpilled) {memoryBytesSpilledQuantiles} else Nil, + if (stageData.hasBytesSpilled) {diskBytesSpilledQuantiles} else Nil) val quantileHeaders = Seq("Metric", "Min", "25th percentile", "Median", "75th percentile", "Max") @@ -397,26 +433,32 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val inputReadable = maybeInput .map(m => s"${Utils.bytesToString(m.bytesRead)} (${m.readMethod.toString.toLowerCase()})") .getOrElse("") + val inputRecords = maybeInput.map(_.recordsRead.toString).getOrElse("") val maybeOutput = metrics.flatMap(_.outputMetrics) val outputSortable = maybeOutput.map(_.bytesWritten.toString).getOrElse("") val outputReadable = maybeOutput .map(m => s"${Utils.bytesToString(m.bytesWritten)}") .getOrElse("") + val outputRecords = maybeOutput.map(_.recordsWritten.toString).getOrElse("") - val maybeShuffleReadBlockedTime = metrics.flatMap(_.shuffleReadMetrics).map(_.fetchWaitTime) - val shuffleReadBlockedTimeSortable = maybeShuffleReadBlockedTime.map(_.toString).getOrElse("") + val maybeShuffleRead = metrics.flatMap(_.shuffleReadMetrics) + val shuffleReadBlockedTimeSortable = maybeShuffleRead + .map(_.fetchWaitTime.toString).getOrElse("") val shuffleReadBlockedTimeReadable = - maybeShuffleReadBlockedTime.map(ms => UIUtils.formatDuration(ms)).getOrElse("") + maybeShuffleRead.map(ms => UIUtils.formatDuration(ms.fetchWaitTime)).getOrElse("") - val maybeShuffleRead = metrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead) - val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("") - val shuffleReadReadable = maybeShuffleRead.map(Utils.bytesToString).getOrElse("") + val shuffleReadSortable = maybeShuffleRead.map(_.remoteBytesRead.toString).getOrElse("") + val shuffleReadReadable = maybeShuffleRead + .map(m => s"${Utils.bytesToString(m.remoteBytesRead)}").getOrElse("") + val shuffleReadRecords = maybeShuffleRead.map(_.recordsRead.toString).getOrElse("") - val maybeShuffleWrite = - metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleBytesWritten) - val shuffleWriteSortable = maybeShuffleWrite.map(_.toString).getOrElse("") - val shuffleWriteReadable = maybeShuffleWrite.map(Utils.bytesToString).getOrElse("") + val maybeShuffleWrite = metrics.flatMap(_.shuffleWriteMetrics) + val shuffleWriteSortable = maybeShuffleWrite.map(_.shuffleBytesWritten.toString).getOrElse("") + val shuffleWriteReadable = maybeShuffleWrite + .map(m => s"${Utils.bytesToString(m.shuffleBytesWritten)}").getOrElse("") + val shuffleWriteRecords = maybeShuffleWrite + .map(_.shuffleRecordsWritten.toString).getOrElse("") val maybeWriteTime = metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleWriteTime) val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("") @@ -472,12 +514,12 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { }} {if (hasInput) { }} {if (hasOutput) { }} {if (hasShuffleRead) { @@ -486,7 +528,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { {shuffleReadBlockedTimeReadable} }} {if (hasShuffleWrite) { @@ -494,7 +536,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { {writeTimeReadable} }} {if (hasBytesSpilled) { 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 01f7e23212c3d..69aac6c862de5 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 @@ -31,9 +31,13 @@ private[jobs] object UIData { var failedTasks : Int = 0 var succeededTasks : Int = 0 var inputBytes : Long = 0 + var inputRecords : Long = 0 var outputBytes : Long = 0 + var outputRecords : Long = 0 var shuffleRead : Long = 0 + var shuffleReadRecords : Long = 0 var shuffleWrite : Long = 0 + var shuffleWriteRecords : Long = 0 var memoryBytesSpilled : Long = 0 var diskBytesSpilled : Long = 0 } @@ -73,9 +77,13 @@ private[jobs] object UIData { var executorRunTime: Long = _ var inputBytes: Long = _ + var inputRecords: Long = _ var outputBytes: Long = _ + var outputRecords: Long = _ var shuffleReadBytes: Long = _ + var shuffleReadRecords : Long = _ var shuffleWriteBytes: Long = _ + var shuffleWriteRecords: Long = _ var memoryBytesSpilled: Long = _ var diskBytesSpilled: Long = _ @@ -85,6 +93,12 @@ private[jobs] object UIData { var accumulables = new HashMap[Long, AccumulableInfo] var taskData = new HashMap[Long, TaskUIData] var executorSummary = new HashMap[String, ExecutorSummary] + + def hasInput = inputBytes > 0 + def hasOutput = outputBytes > 0 + def hasShuffleRead = shuffleReadBytes > 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 c8407bbcb780b..b0b545640f5aa 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -293,22 +293,26 @@ private[spark] object JsonProtocol { ("Remote Blocks Fetched" -> shuffleReadMetrics.remoteBlocksFetched) ~ ("Local Blocks Fetched" -> shuffleReadMetrics.localBlocksFetched) ~ ("Fetch Wait Time" -> shuffleReadMetrics.fetchWaitTime) ~ - ("Remote Bytes Read" -> shuffleReadMetrics.remoteBytesRead) + ("Remote Bytes Read" -> shuffleReadMetrics.remoteBytesRead) ~ + ("Total Records Read" -> shuffleReadMetrics.recordsRead) } def shuffleWriteMetricsToJson(shuffleWriteMetrics: ShuffleWriteMetrics): JValue = { ("Shuffle Bytes Written" -> shuffleWriteMetrics.shuffleBytesWritten) ~ - ("Shuffle Write Time" -> shuffleWriteMetrics.shuffleWriteTime) + ("Shuffle Write Time" -> shuffleWriteMetrics.shuffleWriteTime) ~ + ("Shuffle Records Written" -> shuffleWriteMetrics.shuffleRecordsWritten) } def inputMetricsToJson(inputMetrics: InputMetrics): JValue = { ("Data Read Method" -> inputMetrics.readMethod.toString) ~ - ("Bytes Read" -> inputMetrics.bytesRead) + ("Bytes Read" -> inputMetrics.bytesRead) ~ + ("Records Read" -> inputMetrics.recordsRead) } def outputMetricsToJson(outputMetrics: OutputMetrics): JValue = { ("Data Write Method" -> outputMetrics.writeMethod.toString) ~ - ("Bytes Written" -> outputMetrics.bytesWritten) + ("Bytes Written" -> outputMetrics.bytesWritten) ~ + ("Records Written" -> outputMetrics.recordsWritten) } def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { @@ -670,6 +674,7 @@ 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.incRecordsRead((json \ "Total Records Read").extractOpt[Long].getOrElse(0)) metrics } @@ -677,13 +682,16 @@ private[spark] object JsonProtocol { val metrics = new ShuffleWriteMetrics metrics.incShuffleBytesWritten((json \ "Shuffle Bytes Written").extract[Long]) metrics.incShuffleWriteTime((json \ "Shuffle Write Time").extract[Long]) + metrics.setShuffleRecordsWritten((json \ "Shuffle Records Written") + .extractOpt[Long].getOrElse(0)) metrics } def inputMetricsFromJson(json: JValue): InputMetrics = { val metrics = new InputMetrics( DataReadMethod.withName((json \ "Data Read Method").extract[String])) - metrics.addBytesRead((json \ "Bytes Read").extract[Long]) + metrics.incBytesRead((json \ "Bytes Read").extract[Long]) + metrics.incRecordsRead((json \ "Records Read").extractOpt[Long].getOrElse(0)) metrics } @@ -691,6 +699,7 @@ private[spark] object JsonProtocol { val metrics = new OutputMetrics( DataWriteMethod.withName((json \ "Data Write Method").extract[String])) metrics.setBytesWritten((json \ "Bytes Written").extract[Long]) + metrics.setRecordsWritten((json \ "Records Written").extractOpt[Long].getOrElse(0)) metrics } 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 6ba03841f746b..eaec5a71e6819 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 @@ -763,6 +763,7 @@ private[spark] class ExternalSorter[K, V, C]( if (curWriteMetrics != null) { m.incShuffleBytesWritten(curWriteMetrics.shuffleBytesWritten) m.incShuffleWriteTime(curWriteMetrics.shuffleWriteTime) + m.incShuffleRecordsWritten(curWriteMetrics.shuffleRecordsWritten) } } 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 81db66ae17464..78fa98a3b9065 100644 --- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala @@ -21,44 +21,46 @@ import java.io.{File, FileWriter, PrintWriter} import scala.collection.mutable.ArrayBuffer -import org.scalatest.FunSuite - +import org.apache.commons.lang.math.RandomUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.io.{LongWritable, Text} -import org.apache.hadoop.mapred.{FileSplit => OldFileSplit, InputSplit => OldInputSplit, JobConf, - LineRecordReader => OldLineRecordReader, RecordReader => OldRecordReader, Reporter, - TextInputFormat => OldTextInputFormat} import org.apache.hadoop.mapred.lib.{CombineFileInputFormat => OldCombineFileInputFormat, - CombineFileSplit => OldCombineFileSplit, CombineFileRecordReader => OldCombineFileRecordReader} -import org.apache.hadoop.mapreduce.{InputSplit => NewInputSplit, RecordReader => NewRecordReader, - TaskAttemptContext} + CombineFileRecordReader => OldCombineFileRecordReader, CombineFileSplit => OldCombineFileSplit} +import org.apache.hadoop.mapred.{JobConf, Reporter, FileSplit => OldFileSplit, + InputSplit => OldInputSplit, LineRecordReader => OldLineRecordReader, + RecordReader => OldRecordReader, TextInputFormat => OldTextInputFormat} import org.apache.hadoop.mapreduce.lib.input.{CombineFileInputFormat => NewCombineFileInputFormat, CombineFileRecordReader => NewCombineFileRecordReader, CombineFileSplit => NewCombineFileSplit, FileSplit => NewFileSplit, TextInputFormat => NewTextInputFormat} +import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} +import org.apache.hadoop.mapreduce.{TaskAttemptContext, InputSplit => NewInputSplit, + RecordReader => NewRecordReader} +import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark.SharedSparkContext import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} import org.apache.spark.util.Utils -class InputOutputMetricsSuite extends FunSuite with SharedSparkContext { +class InputOutputMetricsSuite extends FunSuite with SharedSparkContext + with BeforeAndAfter { @transient var tmpDir: File = _ @transient var tmpFile: File = _ @transient var tmpFilePath: String = _ + @transient val numRecords: Int = 100000 + @transient val numBuckets: Int = 10 - override def beforeAll() { - super.beforeAll() - + before { tmpDir = Utils.createTempDir() val testTempDir = new File(tmpDir, "test") testTempDir.mkdir() tmpFile = new File(testTempDir, getClass.getSimpleName + ".txt") val pw = new PrintWriter(new FileWriter(tmpFile)) - for (x <- 1 to 1000000) { - pw.println("s") + for (x <- 1 to numRecords) { + pw.println(RandomUtils.nextInt(numBuckets)) } pw.close() @@ -66,8 +68,7 @@ class InputOutputMetricsSuite extends FunSuite with SharedSparkContext { tmpFilePath = "file://" + tmpFile.getAbsolutePath } - override def afterAll() { - super.afterAll() + after { Utils.deleteRecursively(tmpDir) } @@ -155,6 +156,101 @@ class InputOutputMetricsSuite extends FunSuite with SharedSparkContext { assert(bytesRead >= tmpFile.length()) } + test("input metrics on records read - simple") { + val records = runAndReturnRecordsRead { + sc.textFile(tmpFilePath, 4).count() + } + assert(records == numRecords) + } + + test("input metrics on records read - more stages") { + val records = runAndReturnRecordsRead { + sc.textFile(tmpFilePath, 4) + .map(key => (key.length, 1)) + .reduceByKey(_ + _) + .count() + } + assert(records == numRecords) + } + + test("input metrics on records - New Hadoop API") { + val records = runAndReturnRecordsRead { + sc.newAPIHadoopFile(tmpFilePath, classOf[NewTextInputFormat], classOf[LongWritable], + classOf[Text]).count() + } + assert(records == numRecords) + } + + test("input metrics on recordsd read with cache") { + // prime the cache manager + val rdd = sc.textFile(tmpFilePath, 4).cache() + rdd.collect() + + val records = runAndReturnRecordsRead { + rdd.count() + } + + assert(records == numRecords) + } + + test("shuffle records read metrics") { + val recordsRead = runAndReturnShuffleRecordsRead { + sc.textFile(tmpFilePath, 4) + .map(key => (key, 1)) + .groupByKey() + .collect() + } + assert(recordsRead == numRecords) + } + + test("shuffle records written metrics") { + val recordsWritten = runAndReturnShuffleRecordsWritten { + sc.textFile(tmpFilePath, 4) + .map(key => (key, 1)) + .groupByKey() + .collect() + } + assert(recordsWritten == numRecords) + } + + /** + * Tests the metrics from end to end. + * 1) reading a hadoop file + * 2) shuffle and writing to a hadoop file. + * 3) writing to hadoop file. + */ + test("input read/write and shuffle read/write metrics all line up") { + var inputRead = 0L + var outputWritten = 0L + var shuffleRead = 0L + var shuffleWritten = 0L + sc.addSparkListener(new SparkListener() { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val metrics = taskEnd.taskMetrics + metrics.inputMetrics.foreach(inputRead += _.recordsRead) + metrics.outputMetrics.foreach(outputWritten += _.recordsWritten) + metrics.shuffleReadMetrics.foreach(shuffleRead += _.recordsRead) + metrics.shuffleWriteMetrics.foreach(shuffleWritten += _.shuffleRecordsWritten) + } + }) + + val tmpFile = new File(tmpDir, getClass.getSimpleName) + + sc.textFile(tmpFilePath, 4) + .map(key => (key, 1)) + .reduceByKey(_+_) + .saveAsTextFile("file://" + tmpFile.getAbsolutePath) + + sc.listenerBus.waitUntilEmpty(500) + assert(inputRead == numRecords) + + // Only supported on newer Hadoop + if (SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback().isDefined) { + assert(outputWritten == numBuckets) + } + assert(shuffleRead == shuffleWritten) + } + test("input metrics with interleaved reads") { val numPartitions = 2 val cartVector = 0 to 9 @@ -193,18 +289,66 @@ class InputOutputMetricsSuite extends FunSuite with SharedSparkContext { assert(cartesianBytes == firstSize * numPartitions + (cartVector.length * secondSize)) } - private def runAndReturnBytesRead(job : => Unit): Long = { - val taskBytesRead = new ArrayBuffer[Long]() + private def runAndReturnBytesRead(job: => Unit): Long = { + runAndReturnMetrics(job, _.taskMetrics.inputMetrics.map(_.bytesRead)) + } + + private def runAndReturnRecordsRead(job: => Unit): Long = { + runAndReturnMetrics(job, _.taskMetrics.inputMetrics.map(_.recordsRead)) + } + + private def runAndReturnRecordsWritten(job: => Unit): Long = { + runAndReturnMetrics(job, _.taskMetrics.outputMetrics.map(_.recordsWritten)) + } + + private def runAndReturnShuffleRecordsRead(job: => Unit): Long = { + runAndReturnMetrics(job, _.taskMetrics.shuffleReadMetrics.map(_.recordsRead)) + } + + private def runAndReturnShuffleRecordsWritten(job: => Unit): Long = { + runAndReturnMetrics(job, _.taskMetrics.shuffleWriteMetrics.map(_.shuffleRecordsWritten)) + } + + private def runAndReturnMetrics(job: => Unit, + collector: (SparkListenerTaskEnd) => Option[Long]): Long = { + val taskMetrics = new ArrayBuffer[Long]() sc.addSparkListener(new SparkListener() { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - taskBytesRead += taskEnd.taskMetrics.inputMetrics.get.bytesRead + collector(taskEnd).foreach(taskMetrics += _) } }) job sc.listenerBus.waitUntilEmpty(500) - taskBytesRead.sum + taskMetrics.sum + } + + test("output metrics on records written") { + // Only supported on newer Hadoop + if (SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback().isDefined) { + val file = new File(tmpDir, getClass.getSimpleName) + val filePath = "file://" + file.getAbsolutePath + + val records = runAndReturnRecordsWritten { + sc.parallelize(1 to numRecords).saveAsTextFile(filePath) + } + assert(records == numRecords) + } + } + + test("output metrics on records written - new Hadoop API") { + // Only supported on newer Hadoop + if (SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback().isDefined) { + val file = new File(tmpDir, getClass.getSimpleName) + val filePath = "file://" + file.getAbsolutePath + + val records = runAndReturnRecordsWritten { + sc.parallelize(1 to numRecords).map(key => (key.toString, key.toString)) + .saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](filePath) + } + assert(records == numRecords) + } } test("output metrics when writing text file") { @@ -318,4 +462,4 @@ class NewCombineTextRecordReaderWrapper( override def getCurrentValue(): Text = delegate.getCurrentValue override def getProgress(): Float = delegate.getProgress override def close(): Unit = delegate.close() -} \ No newline at end of file +} 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 bbc7e1357b90d..c21c92b63ad13 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala @@ -31,6 +31,8 @@ class BlockObjectWriterSuite extends FunSuite { new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) writer.write(Long.box(20)) + // Record metrics update on every write + assert(writeMetrics.shuffleRecordsWritten === 1) // Metrics don't update on every write assert(writeMetrics.shuffleBytesWritten == 0) // After 32 writes, metrics should update @@ -39,6 +41,7 @@ class BlockObjectWriterSuite extends FunSuite { writer.write(Long.box(i)) } assert(writeMetrics.shuffleBytesWritten > 0) + assert(writeMetrics.shuffleRecordsWritten === 33) writer.commitAndClose() assert(file.length() == writeMetrics.shuffleBytesWritten) } @@ -51,6 +54,8 @@ class BlockObjectWriterSuite extends FunSuite { new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) writer.write(Long.box(20)) + // Record metrics update on every write + assert(writeMetrics.shuffleRecordsWritten === 1) // Metrics don't update on every write assert(writeMetrics.shuffleBytesWritten == 0) // After 32 writes, metrics should update @@ -59,7 +64,23 @@ class BlockObjectWriterSuite extends FunSuite { writer.write(Long.box(i)) } assert(writeMetrics.shuffleBytesWritten > 0) + assert(writeMetrics.shuffleRecordsWritten === 33) writer.revertPartialWritesAndClose() assert(writeMetrics.shuffleBytesWritten == 0) + assert(writeMetrics.shuffleRecordsWritten == 0) + } + + test("Reopening a closed block writer") { + val file = new File("somefile") + file.deleteOnExit() + val writeMetrics = new ShuffleWriteMetrics() + val writer = new DiskBlockObjectWriter(new TestBlockId("0"), file, + new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) + + writer.open() + writer.close() + intercept[IllegalStateException] { + writer.open() + } } } 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 68074ae32a672..e8405baa8e3ea 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 @@ -234,7 +234,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc taskMetrics.incMemoryBytesSpilled(base + 6) val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) taskMetrics.setInputMetrics(Some(inputMetrics)) - inputMetrics.addBytesRead(base + 7) + inputMetrics.incBytesRead(base + 7) val outputMetrics = new OutputMetrics(DataWriteMethod.Hadoop) taskMetrics.outputMetrics = Some(outputMetrics) outputMetrics.setBytesWritten(base + 8) 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 842f54529baf0..f3017dc42cd5c 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -189,6 +189,34 @@ class JsonProtocolSuite extends FunSuite { assert(newMetrics.inputMetrics.isEmpty) } + test("Input/Output records backwards compatibility") { + // records read were added after 1.2 + val metrics = makeTaskMetrics(1L, 2L, 3L, 4L, 5, 6, + hasHadoopInput = true, hasOutput = true, hasRecords = false) + assert(metrics.inputMetrics.nonEmpty) + assert(metrics.outputMetrics.nonEmpty) + val newJson = JsonProtocol.taskMetricsToJson(metrics) + val oldJson = newJson.removeField { case (field, _) => field == "Records Read" } + .removeField { case (field, _) => field == "Records Written" } + val newMetrics = JsonProtocol.taskMetricsFromJson(oldJson) + assert(newMetrics.inputMetrics.get.recordsRead == 0) + assert(newMetrics.outputMetrics.get.recordsWritten == 0) + } + + test("Shuffle Read/Write records backwards compatibility") { + // records read were added after 1.2 + val metrics = makeTaskMetrics(1L, 2L, 3L, 4L, 5, 6, + hasHadoopInput = false, hasOutput = false, hasRecords = false) + assert(metrics.shuffleReadMetrics.nonEmpty) + assert(metrics.shuffleWriteMetrics.nonEmpty) + val newJson = JsonProtocol.taskMetricsToJson(metrics) + val oldJson = newJson.removeField { case (field, _) => field == "Total Records Read" } + .removeField { case (field, _) => field == "Shuffle Records Written" } + val newMetrics = JsonProtocol.taskMetricsFromJson(oldJson) + assert(newMetrics.shuffleReadMetrics.get.recordsRead == 0) + assert(newMetrics.shuffleWriteMetrics.get.shuffleRecordsWritten == 0) + } + test("OutputMetrics backward compatibility") { // OutputMetrics were added after 1.1 val metrics = makeTaskMetrics(1L, 2L, 3L, 4L, 5, 6, hasHadoopInput = false, hasOutput = true) @@ -644,7 +672,8 @@ class JsonProtocolSuite extends FunSuite { e: Int, f: Int, hasHadoopInput: Boolean, - hasOutput: Boolean) = { + hasOutput: Boolean, + hasRecords: Boolean = true) = { val t = new TaskMetrics t.setHostname("localhost") t.setExecutorDeserializeTime(a) @@ -656,7 +685,8 @@ class JsonProtocolSuite extends FunSuite { if (hasHadoopInput) { val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) - inputMetrics.addBytesRead(d + e + f) + inputMetrics.incBytesRead(d + e + f) + inputMetrics.incRecordsRead(if (hasRecords) (d + e + f) / 100 else -1) t.setInputMetrics(Some(inputMetrics)) } else { val sr = new ShuffleReadMetrics @@ -664,16 +694,19 @@ class JsonProtocolSuite extends FunSuite { sr.incLocalBlocksFetched(e) sr.incFetchWaitTime(a + d) sr.incRemoteBlocksFetched(f) + sr.incRecordsRead(if (hasRecords) (b + d) / 100 else -1) t.setShuffleReadMetrics(Some(sr)) } if (hasOutput) { val outputMetrics = new OutputMetrics(DataWriteMethod.Hadoop) outputMetrics.setBytesWritten(a + b + c) + outputMetrics.setRecordsWritten(if (hasRecords) (a + b + c)/100 else -1) t.outputMetrics = Some(outputMetrics) } else { val sw = new ShuffleWriteMetrics sw.incShuffleBytesWritten(a + b + c) sw.incShuffleWriteTime(b + c + d) + sw.setShuffleRecordsWritten(if (hasRecords) (a + b + c) / 100 else -1) t.shuffleWriteMetrics = Some(sw) } // Make at most 6 blocks @@ -907,11 +940,13 @@ class JsonProtocolSuite extends FunSuite { | "Remote Blocks Fetched": 800, | "Local Blocks Fetched": 700, | "Fetch Wait Time": 900, - | "Remote Bytes Read": 1000 + | "Remote Bytes Read": 1000, + | "Total Records Read" : 10 | }, | "Shuffle Write Metrics": { | "Shuffle Bytes Written": 1200, - | "Shuffle Write Time": 1500 + | "Shuffle Write Time": 1500, + | "Shuffle Records Written": 12 | }, | "Updated Blocks": [ | { @@ -988,11 +1023,13 @@ class JsonProtocolSuite extends FunSuite { | "Disk Bytes Spilled": 0, | "Shuffle Write Metrics": { | "Shuffle Bytes Written": 1200, - | "Shuffle Write Time": 1500 + | "Shuffle Write Time": 1500, + | "Shuffle Records Written": 12 | }, | "Input Metrics": { | "Data Read Method": "Hadoop", - | "Bytes Read": 2100 + | "Bytes Read": 2100, + | "Records Read": 21 | }, | "Updated Blocks": [ | { @@ -1069,11 +1106,13 @@ class JsonProtocolSuite extends FunSuite { | "Disk Bytes Spilled": 0, | "Input Metrics": { | "Data Read Method": "Hadoop", - | "Bytes Read": 2100 + | "Bytes Read": 2100, + | "Records Read": 21 | }, | "Output Metrics": { | "Data Write Method": "Hadoop", - | "Bytes Written": 1200 + | "Bytes Written": 1200, + | "Records Written": 12 | }, | "Updated Blocks": [ | { From 2bda1c1d376afd8abe6a04be345461752f3fb1b6 Mon Sep 17 00:00:00 2001 From: huangzhaowei Date: Fri, 6 Feb 2015 14:35:29 -0800 Subject: [PATCH 147/578] [SPARK-5444][Network]Add a retry to deal with the conflict port in netty server. If the `spark.blockMnager.port` had conflicted with a specific port, Spark will throw an exception and exit. So add a retry to avoid this situation. Author: huangzhaowei Closes #4240 from SaintBacchus/NettyPortConflict and squashes the following commits: cc926d2 [huangzhaowei] Add a retry to deal with the conflict port in netty server. --- .../spark/network/server/TransportServer.java | 36 +++++++++++++++++-- .../spark/network/util/TransportConf.java | 7 ++++ 2 files changed, 41 insertions(+), 2 deletions(-) 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 625c3257d764e..ef209991804b4 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 @@ -100,8 +100,7 @@ protected void initChannel(SocketChannel ch) throws Exception { } }); - channelFuture = bootstrap.bind(new InetSocketAddress(portToBind)); - channelFuture.syncUninterruptibly(); + bindRightPort(portToBind); port = ((InetSocketAddress) channelFuture.channel().localAddress()).getPort(); logger.debug("Shuffle server started on port :" + port); @@ -123,4 +122,37 @@ public void close() { bootstrap = null; } + /** + * Attempt to bind to the specified port up to a fixed number of retries. + * If all attempts fail after the max number of retries, exit. + */ + private void bindRightPort(int portToBind) { + int maxPortRetries = conf.portMaxRetries(); + + for (int i = 0; i <= maxPortRetries; i++) { + int tryPort = -1; + if (0 == portToBind) { + // Do not increment port if tryPort is 0, which is treated as a special port + tryPort = 0; + } else { + // If the new port wraps around, do not try a privilege port + tryPort = ((portToBind + i - 1024) % (65536 - 1024)) + 1024; + } + try { + channelFuture = bootstrap.bind(new InetSocketAddress(tryPort)); + channelFuture.syncUninterruptibly(); + return; + } catch (Exception e) { + logger.warn("Netty service could not bind on port " + tryPort + + ". Attempting the next port."); + if (i >= maxPortRetries) { + logger.error(e.getMessage() + ": Netty server failed after " + + maxPortRetries + " retries."); + + // If it can't find a right port, it should exit directly. + System.exit(-1); + } + } + } + } } 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 6c9178688693f..2eaf3b71d9a49 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 @@ -98,4 +98,11 @@ public int memoryMapBytes() { public boolean lazyFileDescriptor() { return conf.getBoolean("spark.shuffle.io.lazyFD", true); } + + /** + * Maximum number of retries when binding to a port before giving up. + */ + public int portMaxRetries() { + return conf.getInt("spark.port.maxRetries", 16); + } } From 61073f832128845a76469fc37376483b784c927b Mon Sep 17 00:00:00 2001 From: lianhuiwang Date: Fri, 6 Feb 2015 14:47:52 -0800 Subject: [PATCH 148/578] [SPARK-4994][network]Cleanup removed executors' ShuffleInfo in yarn shuffle service when the application is completed, yarn's nodemanager can remove application's local-dirs.but all executors' metadata of completed application havenot be removed. now it lets yarn ShuffleService to have much more memory to store Executors' ShuffleInfo. so these metadata need to be removed. Author: lianhuiwang Closes #3828 from lianhuiwang/SPARK-4994 and squashes the following commits: f3ba1d2 [lianhuiwang] Cleanup removed executors' ShuffleInfo --- .../org/apache/spark/network/yarn/YarnShuffleService.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index a34aabe9e78a6..63b21222e7b77 100644 --- a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -76,6 +76,9 @@ public class YarnShuffleService extends AuxiliaryService { // The actual server that serves shuffle files private TransportServer shuffleServer = null; + // Handles registering executors and opening shuffle blocks + private ExternalShuffleBlockHandler blockHandler; + public YarnShuffleService() { super("spark_shuffle"); logger.info("Initializing YARN shuffle service for Spark"); @@ -99,7 +102,8 @@ protected void serviceInit(Configuration conf) { // If authentication is enabled, set up the shuffle server to use a // special RPC handler that filters out unauthenticated fetch requests boolean authEnabled = conf.getBoolean(SPARK_AUTHENTICATE_KEY, DEFAULT_SPARK_AUTHENTICATE); - RpcHandler rpcHandler = new ExternalShuffleBlockHandler(transportConf); + blockHandler = new ExternalShuffleBlockHandler(transportConf); + RpcHandler rpcHandler = blockHandler; if (authEnabled) { secretManager = new ShuffleSecretManager(); rpcHandler = new SaslRpcHandler(rpcHandler, secretManager); @@ -136,6 +140,7 @@ public void stopApplication(ApplicationTerminationContext context) { if (isAuthenticationEnabled()) { secretManager.unregisterApp(appId); } + blockHandler.applicationRemoved(appId, false /* clean up local dirs */); } catch (Exception e) { logger.error("Exception when stopping application {}", appId, e); } From 76c4bf59f6544b9c02d99fd18436427cbce632e6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 6 Feb 2015 15:11:02 -0800 Subject: [PATCH 149/578] [SQL] Use TestSQLContext in Java tests Sometimes tests were failing due to the creation of multiple `SparkContext`s in a single JVM. Author: Michael Armbrust Closes #4441 from marmbrus/javaTests and squashes the following commits: 657b1e0 [Michael Armbrust] [SQL] Use TestSQLContext in Java tests --- .../java/org/apache/spark/sql/api/java/JavaAPISuite.java | 7 +++---- .../apache/spark/sql/api/java/JavaApplySchemaSuite.java | 6 +++--- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java index e5588938ea162..a21a15409080c 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java @@ -19,6 +19,7 @@ import java.io.Serializable; +import org.apache.spark.sql.test.TestSQLContext$; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -37,14 +38,12 @@ public class JavaAPISuite implements Serializable { @Before public void setUp() { - sc = new JavaSparkContext("local", "JavaAPISuite"); - sqlContext = new SQLContext(sc); + sqlContext = TestSQLContext$.MODULE$; + sc = new JavaSparkContext(sqlContext.sparkContext()); } @After public void tearDown() { - sc.stop(); - sc = null; } @SuppressWarnings("unchecked") 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 8510bac499092..2e6e977fdc752 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 @@ -22,6 +22,7 @@ import java.util.Arrays; import java.util.List; +import org.apache.spark.sql.test.TestSQLContext$; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -42,13 +43,12 @@ public class JavaApplySchemaSuite implements Serializable { @Before public void setUp() { - javaCtx = new JavaSparkContext("local", "JavaApplySchemaSuite"); - javaSqlCtx = new SQLContext(javaCtx); + javaSqlCtx = TestSQLContext$.MODULE$; + javaCtx = new JavaSparkContext(javaSqlCtx.sparkContext()); } @After public void tearDown() { - javaCtx.stop(); javaCtx = null; javaSqlCtx = null; } From c4021401e326cd5a412a70425f5c75405284880e Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 6 Feb 2015 15:23:42 -0800 Subject: [PATCH 150/578] [SQL] [Minor] HiveParquetSuite was disabled by mistake, re-enable them [Review on Reviewable](https://reviewable.io/reviews/apache/spark/4440) Author: Cheng Lian Closes #4440 from liancheng/parquet-oops and squashes the following commits: f21ede4 [Cheng Lian] HiveParquetSuite was disabled by mistake, re-enable them. --- .../spark/sql/parquet/ParquetQuerySuite.scala | 1 - .../apache/spark/sql/parquet/HiveParquetSuite.scala | 13 ++++++++++--- 2 files changed, 10 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 48c7598343e55..cba06835f9a61 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 @@ -35,7 +35,6 @@ class ParquetQuerySuite extends QueryTest with ParquetTest { } } - // TODO Re-enable this after data source insertion API is merged test(s"$prefix: appending") { val data = (0 until 10).map(i => (i, i.toString)) withParquetTable(data, "t") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala index eae69af5864aa..e89b4489f15d1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.parquet -import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.{SQLConf, QueryTest} import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.hive.test.TestHive @@ -64,8 +64,7 @@ class HiveParquetSuite extends QueryTest with ParquetTest { } } - // TODO Re-enable this after data source insertion API is merged - ignore(s"$prefix: INSERT OVERWRITE TABLE Parquet table") { + test(s"$prefix: INSERT OVERWRITE TABLE Parquet table") { withParquetTable((1 to 10).map(i => (i, s"val_$i")), "t") { withTempPath { file => sql("SELECT * FROM t LIMIT 1").saveAsParquetFile(file.getCanonicalPath) @@ -81,4 +80,12 @@ class HiveParquetSuite extends QueryTest with ParquetTest { } } } + + withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API -> "true") { + run("Parquet data source enabled") + } + + withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API -> "false") { + run("Parquet data source disabled") + } } From 0e23ca9f805b46d9b3472330676e5c8db926b8f5 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 6 Feb 2015 15:42:59 -0800 Subject: [PATCH 151/578] [SPARK-5601][MLLIB] make streaming linear algorithms Java-friendly Overload `trainOn`, `predictOn`, and `predictOnValues`. CC freeman-lab Author: Xiangrui Meng Closes #4432 from mengxr/streaming-java and squashes the following commits: 6a79b85 [Xiangrui Meng] add java test for streaming logistic regression 2d7b357 [Xiangrui Meng] organize imports 1f662b3 [Xiangrui Meng] make streaming linear algorithms Java-friendly --- .../regression/StreamingLinearAlgorithm.scala | 20 ++++- .../JavaStreamingLogisticRegressionSuite.java | 82 +++++++++++++++++++ .../JavaStreamingLinearRegressionSuite.java | 80 ++++++++++++++++++ 3 files changed, 181 insertions(+), 1 deletion(-) create mode 100644 mllib/src/test/java/org/apache/spark/ml/classification/JavaStreamingLogisticRegressionSuite.java create mode 100644 mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java 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 c854f124451cf..ce95c063db970 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 @@ -21,7 +21,9 @@ import scala.reflect.ClassTag import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.streaming.api.java.{JavaDStream, JavaPairDStream} import org.apache.spark.streaming.dstream.DStream /** @@ -76,7 +78,7 @@ abstract class StreamingLinearAlgorithm[ * * @param data DStream containing labeled data */ - def trainOn(data: DStream[LabeledPoint]) { + def trainOn(data: DStream[LabeledPoint]): Unit = { if (model.isEmpty) { throw new IllegalArgumentException("Model must be initialized before starting training.") } @@ -99,6 +101,9 @@ abstract class StreamingLinearAlgorithm[ } } + /** Java-friendly version of `trainOn`. */ + def trainOn(data: JavaDStream[LabeledPoint]): Unit = trainOn(data.dstream) + /** * Use the model to make predictions on batches of data from a DStream * @@ -112,6 +117,11 @@ abstract class StreamingLinearAlgorithm[ data.map(model.get.predict) } + /** Java-friendly version of `predictOn`. */ + def predictOn(data: JavaDStream[Vector]): JavaDStream[java.lang.Double] = { + JavaDStream.fromDStream(predictOn(data.dstream).asInstanceOf[DStream[java.lang.Double]]) + } + /** * Use the model to make predictions on the values of a DStream and carry over its keys. * @param data DStream containing feature vectors @@ -124,4 +134,12 @@ abstract class StreamingLinearAlgorithm[ } data.mapValues(model.get.predict) } + + + /** Java-friendly version of `predictOnValues`. */ + def predictOnValues[K](data: JavaPairDStream[K, Vector]): JavaPairDStream[K, java.lang.Double] = { + implicit val tag = fakeClassTag[K] + JavaPairDStream.fromPairDStream( + predictOnValues(data.dstream).asInstanceOf[DStream[(K, java.lang.Double)]]) + } } 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 new file mode 100644 index 0000000000000..ac945ba6f23c1 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaStreamingLogisticRegressionSuite.java @@ -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.classification; + +import java.io.Serializable; +import java.util.List; + +import scala.Tuple2; + +import com.google.common.collect.Lists; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.SparkConf; +import org.apache.spark.mllib.classification.StreamingLogisticRegressionWithSGD; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; +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 static org.apache.spark.streaming.JavaTestUtils.*; + +public class JavaStreamingLogisticRegressionSuite implements Serializable { + + protected transient JavaStreamingContext ssc; + + @Before + public void setUp() { + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); + ssc.checkpoint("checkpoint"); + } + + @After + public void tearDown() { + ssc.stop(); + ssc = null; + } + + @Test + @SuppressWarnings("unchecked") + public void javaAPI() { + List trainingBatch = Lists.newArrayList( + new LabeledPoint(1.0, Vectors.dense(1.0)), + new LabeledPoint(0.0, Vectors.dense(0.0))); + JavaDStream training = + attachTestInputStream(ssc, Lists.newArrayList(trainingBatch, trainingBatch), 2); + List> testBatch = Lists.newArrayList( + new Tuple2(10, Vectors.dense(1.0)), + new Tuple2(11, Vectors.dense(0.0))); + JavaPairDStream test = JavaPairDStream.fromJavaDStream( + attachTestInputStream(ssc, Lists.newArrayList(testBatch, testBatch), 2)); + StreamingLogisticRegressionWithSGD slr = new StreamingLogisticRegressionWithSGD() + .setNumIterations(2) + .setInitialWeights(Vectors.dense(0.0)); + slr.trainOn(training); + JavaPairDStream prediction = slr.predictOnValues(test); + attachTestOutputStream(prediction.count()); + runStreams(ssc, 2, 2); + } +} 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 new file mode 100644 index 0000000000000..a4dd1ac39a3c8 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.regression; + +import java.io.Serializable; +import java.util.List; + +import scala.Tuple2; + +import com.google.common.collect.Lists; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.SparkConf; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +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 static org.apache.spark.streaming.JavaTestUtils.*; + +public class JavaStreamingLinearRegressionSuite implements Serializable { + + protected transient JavaStreamingContext ssc; + + @Before + public void setUp() { + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); + ssc.checkpoint("checkpoint"); + } + + @After + public void tearDown() { + ssc.stop(); + ssc = null; + } + + @Test + @SuppressWarnings("unchecked") + public void javaAPI() { + List trainingBatch = Lists.newArrayList( + new LabeledPoint(1.0, Vectors.dense(1.0)), + new LabeledPoint(0.0, Vectors.dense(0.0))); + JavaDStream training = + attachTestInputStream(ssc, Lists.newArrayList(trainingBatch, trainingBatch), 2); + List> testBatch = Lists.newArrayList( + new Tuple2(10, Vectors.dense(1.0)), + new Tuple2(11, Vectors.dense(0.0))); + JavaPairDStream test = JavaPairDStream.fromJavaDStream( + attachTestInputStream(ssc, Lists.newArrayList(testBatch, testBatch), 2)); + StreamingLinearRegressionWithSGD slr = new StreamingLinearRegressionWithSGD() + .setNumIterations(2) + .setInitialWeights(Vectors.dense(0.0)); + slr.trainOn(training); + JavaPairDStream prediction = slr.predictOnValues(test); + attachTestOutputStream(prediction.count()); + runStreams(ssc, 2, 2); + } +} From e772b4e4e1b790199dd000bd096a8917cb8def24 Mon Sep 17 00:00:00 2001 From: Grzegorz Dubicki Date: Fri, 6 Feb 2015 15:43:58 -0800 Subject: [PATCH 152/578] SPARK-5403: Ignore UserKnownHostsFile in SSH calls See https://issues.apache.org/jira/browse/SPARK-5403 Author: Grzegorz Dubicki Closes #4196 from grzegorz-dubicki/SPARK-5403 and squashes the following commits: a7d863f [Grzegorz Dubicki] Resolve start command hanging issue --- ec2/spark_ec2.py | 1 + 1 file changed, 1 insertion(+) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 7371558274d78..3f7242a53d6fd 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -899,6 +899,7 @@ def stringify_command(parts): def ssh_args(opts): parts = ['-o', 'StrictHostKeyChecking=no'] + parts += ['-o', 'UserKnownHostsFile=/dev/null'] if opts.identity_file is not None: parts += ['-i', opts.identity_file] return parts From 1390e56fa86d07b588f0c3a1a2cc25ab5bb0db67 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 6 Feb 2015 15:57:06 -0800 Subject: [PATCH 153/578] [SPARK-5388] Provide a stable application submission gateway for standalone cluster mode The goal is to provide a stable, REST-based application submission gateway that is not inherently based on Akka, which is unstable across versions. This PR targets standalone cluster mode, but is implemented in a general enough manner that can be potentially extended to other modes in the future. Client mode is currently not included in the changes here because there are many more Akka messages exchanged there. As of the changes here, the Master will advertise two ports, 7077 and 6066. We need to keep around the old one (7077) for client mode and older versions of Spark submit. However, all new versions of Spark submit will use the REST gateway (6066). By the way this includes ~700 lines of tests and ~200 lines of license. Author: Andrew Or Closes #4216 from andrewor14/rest and squashes the following commits: 8d7ce07 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest 6f0c597 [Andrew Or] Use nullable fields for integer and boolean values dfe4bd7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest b9e2a08 [Andrew Or] Minor comments 02b5cea [Andrew Or] Fix tests d2b1ef8 [Andrew Or] Comment changes + minor code refactoring across the board 9c82a36 [Andrew Or] Minor comment and wording updates b4695e7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest c9a8ad7 [Andrew Or] Do not include appResource and mainClass as properties 6fc7670 [Andrew Or] Report REST server response back to the user 40e6095 [Andrew Or] Pass submit parameters through system properties cbd670b [Andrew Or] Include unknown fields, if any, in server response 9fee16f [Andrew Or] Include server protocol version on mismatch 09f873a [Andrew Or] Fix style 8188e61 [Andrew Or] Upgrade Jackson from 2.3.0 to 2.4.4 37538e0 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest 9165ae8 [Andrew Or] Fall back to Akka if endpoint was not REST 252d53c [Andrew Or] Clean up server error handling behavior further c643f64 [Andrew Or] Fix style bbbd329 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest 792e112 [Andrew Or] Use specific HTTP response codes on error f98660b [Andrew Or] Version the protocol and include it in REST URL 721819f [Andrew Or] Provide more REST-like interface for submit/kill/status 581f7bf [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest 9e0d1af [Andrew Or] Move some classes around to reduce number of files (minor) 42e5de4 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest 1f1c03f [Andrew Or] Use Jackson's DefaultScalaModule to simplify messages 9229433 [Andrew Or] Reduce duplicate naming in REST field ade28fd [Andrew Or] Clean up REST response output in Spark submit b2fef8b [Andrew Or] Abstract the success field to the general response 6c57b4b [Andrew Or] Increase timeout in end-to-end tests bf696ff [Andrew Or] Add checks for enabling REST when using kill/status 7ee6737 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest e2f7f5f [Andrew Or] Provide more safeguard against missing fields 9581df7 [Andrew Or] Clean up uses of exceptions 914fdff [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest e2104e6 [Andrew Or] stable -> rest 3db7379 [Andrew Or] Fix comments and name fields for better error messages 8d43486 [Andrew Or] Replace SubmitRestProtocolAction with class name df90e8b [Andrew Or] Use Jackson for JSON de/serialization d7a1f9f [Andrew Or] Fix local cluster tests efa5e18 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest e42c131 [Andrew Or] Add end-to-end tests for standalone REST protocol 837475b [Andrew Or] Show the REST port on the Master UI d8d3717 [Andrew Or] Use a daemon thread pool for REST server 6568ca5 [Andrew Or] Merge branch 'master' of github.com:apache/spark into rest 77774ba [Andrew Or] Minor fixes 206cae4 [Andrew Or] Refactor and add tests for the REST protocol 63c05b3 [Andrew Or] Remove MASTER as a field (minor) 9e21b72 [Andrew Or] Action -> SparkSubmitAction (minor) 51c5ca6 [Andrew Or] Distinguish client and server side Spark versions b44e103 [Andrew Or] Implement status requests + fix validation behavior 120ab9d [Andrew Or] Support kill and request driver status through SparkSubmit 544de1d [Andrew Or] Major clean ups in code and comments e958cae [Andrew Or] Supported nested values in messages 484bd21 [Andrew Or] Specify an ordering for fields in SubmitDriverRequestMessage 6ff088d [Andrew Or] Rename classes to generalize REST protocol af9d9cb [Andrew Or] Integrate REST protocol in standalone mode 53e7c0e [Andrew Or] Initial client, server, and all the messages --- core/pom.xml | 8 + .../org/apache/spark/ui/static/webui.css | 6 + .../scala/org/apache/spark/SparkContext.scala | 2 +- .../apache/spark/deploy/ClientArguments.scala | 20 +- .../apache/spark/deploy/DeployMessage.scala | 15 +- .../spark/deploy/LocalSparkCluster.scala | 12 +- .../org/apache/spark/deploy/SparkSubmit.scala | 142 ++++-- .../spark/deploy/SparkSubmitArguments.scala | 81 +++- .../apache/spark/deploy/master/Master.scala | 46 +- .../spark/deploy/master/MasterMessages.scala | 4 +- .../spark/deploy/master/ui/MasterPage.scala | 18 +- .../deploy/rest/StandaloneRestClient.scala | 307 ++++++++++++ .../deploy/rest/StandaloneRestServer.scala | 449 ++++++++++++++++++ .../rest/SubmitRestProtocolException.scala | 36 ++ .../rest/SubmitRestProtocolMessage.scala | 146 ++++++ .../rest/SubmitRestProtocolRequest.scala | 78 +++ .../rest/SubmitRestProtocolResponse.scala | 85 ++++ .../spark/deploy/JsonProtocolSuite.scala | 3 +- .../spark/deploy/SparkSubmitSuite.scala | 57 ++- .../rest/StandaloneRestSubmitSuite.scala | 265 +++++++++++ .../deploy/rest/SubmitRestProtocolSuite.scala | 324 +++++++++++++ .../KryoSerializerDistributedSuite.scala | 6 +- pom.xml | 11 + 23 files changed, 2027 insertions(+), 94 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolException.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.scala create mode 100644 core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala diff --git a/core/pom.xml b/core/pom.xml index 4daaf88147142..66180035e61f1 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -243,6 +243,14 @@ io.dropwizard.metrics metrics-graphite + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.module + jackson-module-scala_2.10 + org.apache.derby derby 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 f23ba9dba167f..68b33b5f0d7c7 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 @@ -103,6 +103,12 @@ span.expand-details { float: right; } +span.rest-uri { + font-size: 10pt; + font-style: italic; + color: gray; +} + pre { font-size: 0.8em; } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 5623587c36fa6..71bdbc9b38ddb 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2110,7 +2110,7 @@ object SparkContext extends Logging { val scheduler = new TaskSchedulerImpl(sc) val localCluster = new LocalSparkCluster( - numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) + numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt, sc.conf) val masterUrls = localCluster.start() val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls) scheduler.initialize(backend) 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 e5873ce724b9f..415bd50591692 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -29,8 +29,7 @@ import org.apache.spark.util.{IntParam, MemoryParam} * Command-line parser for the driver client. */ private[spark] class ClientArguments(args: Array[String]) { - val defaultCores = 1 - val defaultMemory = 512 + import ClientArguments._ var cmd: String = "" // 'launch' or 'kill' var logLevel = Level.WARN @@ -39,9 +38,9 @@ private[spark] class ClientArguments(args: Array[String]) { var master: String = "" var jarUrl: String = "" var mainClass: String = "" - var supervise: Boolean = false - var memory: Int = defaultMemory - var cores: Int = defaultCores + var supervise: Boolean = DEFAULT_SUPERVISE + var memory: Int = DEFAULT_MEMORY + var cores: Int = DEFAULT_CORES private var _driverOptions = ListBuffer[String]() def driverOptions = _driverOptions.toSeq @@ -50,7 +49,7 @@ private[spark] class ClientArguments(args: Array[String]) { parse(args.toList) - def parse(args: List[String]): Unit = args match { + private def parse(args: List[String]): Unit = args match { case ("--cores" | "-c") :: IntParam(value) :: tail => cores = value parse(tail) @@ -106,9 +105,10 @@ private[spark] class ClientArguments(args: Array[String]) { |Usage: DriverClient kill | |Options: - | -c CORES, --cores CORES Number of cores to request (default: $defaultCores) - | -m MEMORY, --memory MEMORY Megabytes of memory to request (default: $defaultMemory) + | -c CORES, --cores CORES Number of cores to request (default: $DEFAULT_CORES) + | -m MEMORY, --memory MEMORY Megabytes of memory to request (default: $DEFAULT_MEMORY) | -s, --supervise Whether to restart the driver on failure + | (default: $DEFAULT_SUPERVISE) | -v, --verbose Print more debugging output """.stripMargin System.err.println(usage) @@ -117,6 +117,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 + def isValidJarUrl(s: String): Boolean = { try { val uri = new URI(s) 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 243d8edb72ed3..7f600d89604a2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -148,15 +148,22 @@ private[deploy] object DeployMessages { // Master to MasterWebUI - case class MasterStateResponse(host: String, port: Int, workers: Array[WorkerInfo], - activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo], - activeDrivers: Array[DriverInfo], completedDrivers: Array[DriverInfo], - status: MasterState) { + case class MasterStateResponse( + host: String, + port: Int, + restPort: Option[Int], + workers: Array[WorkerInfo], + activeApps: Array[ApplicationInfo], + completedApps: Array[ApplicationInfo], + activeDrivers: Array[DriverInfo], + completedDrivers: Array[DriverInfo], + status: MasterState) { Utils.checkHost(host, "Required hostname") assert (port > 0) def uri = "spark://" + host + ":" + port + def restUri: Option[String] = restPort.map { p => "spark://" + host + ":" + p } } // WorkerWebUI to Worker 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 9a7a113c95715..0401b15446a7b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -33,7 +33,11 @@ import org.apache.spark.util.Utils * fault recovery without spinning up a lot of processes. */ private[spark] -class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: Int) +class LocalSparkCluster( + numWorkers: Int, + coresPerWorker: Int, + memoryPerWorker: Int, + conf: SparkConf) extends Logging { private val localHostname = Utils.localHostName() @@ -43,9 +47,11 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I def start(): Array[String] = { logInfo("Starting a local Spark cluster with " + numWorkers + " workers.") + // Disable REST server on Master in this mode unless otherwise specified + val _conf = conf.clone().setIfMissing("spark.master.rest.enabled", "false") + /* Start the Master */ - val conf = new SparkConf(false) - val (masterSystem, masterPort, _) = Master.startSystemAndActor(localHostname, 0, 0, conf) + val (masterSystem, masterPort, _, _) = Master.startSystemAndActor(localHostname, 0, 0, _conf) masterActorSystems += masterSystem val masterUrl = "spark://" + localHostname + ":" + masterPort val masters = Array(masterUrl) 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 9d25e647f1703..6d213926f3d7b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -18,25 +18,35 @@ package org.apache.spark.deploy import java.io.{File, PrintStream} -import java.lang.reflect.{Modifier, InvocationTargetException} +import java.lang.reflect.{InvocationTargetException, Modifier} import java.net.URL + import scala.collection.mutable.{ArrayBuffer, HashMap, Map} import org.apache.hadoop.fs.Path import org.apache.ivy.Ivy import org.apache.ivy.core.LogOptions -import org.apache.ivy.core.module.descriptor.{DefaultExcludeRule, DefaultDependencyDescriptor, DefaultModuleDescriptor} -import org.apache.ivy.core.module.id.{ModuleId, ArtifactId, ModuleRevisionId} +import org.apache.ivy.core.module.descriptor._ +import org.apache.ivy.core.module.id.{ArtifactId, ModuleId, ModuleRevisionId} import org.apache.ivy.core.report.ResolveReport -import org.apache.ivy.core.resolve.{IvyNode, ResolveOptions} +import org.apache.ivy.core.resolve.ResolveOptions import org.apache.ivy.core.retrieve.RetrieveOptions 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.executor.ExecutorURLClassLoader + +import org.apache.spark.deploy.rest._ +import org.apache.spark.executor._ import org.apache.spark.util.Utils -import org.apache.spark.executor.ChildExecutorURLClassLoader -import org.apache.spark.executor.MutableURLClassLoader + +/** + * 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 { + type SparkSubmitAction = Value + val SUBMIT, KILL, REQUEST_STATUS = Value +} /** * Main gateway of launching a Spark application. @@ -83,21 +93,74 @@ object SparkSubmit { if (appArgs.verbose) { printStream.println(appArgs) } - val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) - launch(childArgs, classpath, sysProps, mainClass, appArgs.verbose) + appArgs.action match { + case SparkSubmitAction.SUBMIT => submit(appArgs) + case SparkSubmitAction.KILL => kill(appArgs) + case SparkSubmitAction.REQUEST_STATUS => requestStatus(appArgs) + } + } + + /** Kill an existing submission using the REST protocol. Standalone cluster mode only. */ + private def kill(args: SparkSubmitArguments): Unit = { + new StandaloneRestClient() + .killSubmission(args.master, args.submissionToKill) } /** - * @return a tuple containing - * (1) the arguments for the child process, - * (2) a list of classpath entries for the child, - * (3) a list of system properties and env vars, and - * (4) the main class for the child + * Request the status of an existing submission using the REST protocol. + * Standalone cluster mode only. */ - private[spark] def createLaunchEnv(args: SparkSubmitArguments) - : (ArrayBuffer[String], ArrayBuffer[String], Map[String, String], String) = { + private def requestStatus(args: SparkSubmitArguments): Unit = { + new StandaloneRestClient() + .requestSubmissionStatus(args.master, args.submissionToRequestStatusFor) + } - // Values to return + /** + * Submit the application using the provided parameters. + * + * This runs in two steps. First, we prepare the launch environment by setting up + * the appropriate classpath, system properties, and application arguments for + * running the child main class based on the cluster manager and the deploy mode. + * Second, we use this launch environment to invoke the main method of the child + * main class. + */ + private[spark] def submit(args: SparkSubmitArguments): Unit = { + val (childArgs, childClasspath, sysProps, childMainClass) = prepareSubmitEnvironment(args) + // 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 + // The latter is the default behavior as of Spark 1.3, but Spark submit will fail over + // to use the legacy gateway if the master endpoint turns out to be not a REST server. + if (args.isStandaloneCluster && args.useRest) { + try { + printStream.println("Running Spark using the REST application submission protocol.") + runMain(childArgs, childClasspath, sysProps, childMainClass) + } catch { + // Fail over to use the legacy submission gateway + case e: SubmitRestConnectionException => + printWarning(s"Master endpoint ${args.master} was not a REST server. " + + "Falling back to legacy submission gateway instead.") + args.useRest = false + submit(args) + } + // In all other modes, just run the main class as prepared + } else { + runMain(childArgs, childClasspath, sysProps, childMainClass) + } + } + + /** + * Prepare the environment for submitting an application. + * This returns a 4-tuple: + * (1) the arguments for the child process, + * (2) a list of classpath entries for the child, + * (3) a map of system properties, and + * (4) the main class for the child + * Exposed for testing. + */ + private[spark] def prepareSubmitEnvironment(args: SparkSubmitArguments) + : (Seq[String], Seq[String], Map[String, String], String) = { + // Return values val childArgs = new ArrayBuffer[String]() val childClasspath = new ArrayBuffer[String]() val sysProps = new HashMap[String, String]() @@ -235,10 +298,13 @@ object SparkSubmit { sysProp = "spark.driver.extraLibraryPath"), // Standalone cluster only + // Do not set CL arguments here because there are multiple possibilities for the main class OptionAssigner(args.jars, STANDALONE, CLUSTER, sysProp = "spark.jars"), OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, sysProp = "spark.jars.ivy"), - OptionAssigner(args.driverMemory, STANDALONE, CLUSTER, clOption = "--memory"), - OptionAssigner(args.driverCores, STANDALONE, CLUSTER, clOption = "--cores"), + OptionAssigner(args.driverMemory, STANDALONE, CLUSTER, sysProp = "spark.driver.memory"), + OptionAssigner(args.driverCores, STANDALONE, CLUSTER, sysProp = "spark.driver.cores"), + OptionAssigner(args.supervise.toString, STANDALONE, CLUSTER, + sysProp = "spark.driver.supervise"), // Yarn client only OptionAssigner(args.queue, YARN, CLIENT, sysProp = "spark.yarn.queue"), @@ -279,7 +345,6 @@ object SparkSubmit { if (args.childArgs != null) { childArgs ++= args.childArgs } } - // Map all arguments to command-line options or system properties for our chosen mode for (opt <- options) { if (opt.value != null && @@ -301,14 +366,21 @@ object SparkSubmit { sysProps.put("spark.jars", jars.mkString(",")) } - // In standalone-cluster mode, use Client as a wrapper around the user class - if (clusterManager == STANDALONE && deployMode == CLUSTER) { - childMainClass = "org.apache.spark.deploy.Client" - if (args.supervise) { - childArgs += "--supervise" + // In standalone cluster mode, use the REST client to submit the application (Spark 1.3+). + // All Spark parameters are expected to be passed to the client through system properties. + if (args.isStandaloneCluster) { + if (args.useRest) { + childMainClass = "org.apache.spark.deploy.rest.StandaloneRestClient" + childArgs += (args.primaryResource, args.mainClass) + } else { + // In legacy standalone cluster mode, use Client as a wrapper around the user class + childMainClass = "org.apache.spark.deploy.Client" + if (args.supervise) { childArgs += "--supervise" } + Option(args.driverMemory).foreach { m => childArgs += ("--memory", m) } + Option(args.driverCores).foreach { c => childArgs += ("--cores", c) } + childArgs += "launch" + childArgs += (args.master, args.primaryResource, args.mainClass) } - childArgs += "launch" - childArgs += (args.master, args.primaryResource, args.mainClass) if (args.childArgs != null) { childArgs ++= args.childArgs } @@ -345,7 +417,7 @@ object SparkSubmit { // Ignore invalid spark.driver.host in cluster modes. if (deployMode == CLUSTER) { - sysProps -= ("spark.driver.host") + sysProps -= "spark.driver.host" } // Resolve paths in certain spark properties @@ -374,9 +446,15 @@ object SparkSubmit { (childArgs, childClasspath, sysProps, childMainClass) } - private def launch( - childArgs: ArrayBuffer[String], - childClasspath: ArrayBuffer[String], + /** + * Run the main method of the child class using the provided launch environment. + * + * Note that this main class will not be the one provided by the user if we're + * running cluster deploy mode or python applications. + */ + private def runMain( + childArgs: Seq[String], + childClasspath: Seq[String], sysProps: Map[String, String], childMainClass: String, verbose: Boolean = false) { @@ -697,7 +775,7 @@ private[spark] object SparkSubmitUtils { * Provides an indirection layer for passing arguments as system properties or flags to * the user's driver program or to downstream launcher tools. */ -private[spark] case class OptionAssigner( +private case class OptionAssigner( value: String, clusterManager: Int, deployMode: Int, 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 5cadc534f4baa..bd0ae26fd8210 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -22,6 +22,7 @@ import java.util.jar.JarFile import scala.collection.mutable.{ArrayBuffer, HashMap} +import org.apache.spark.deploy.SparkSubmitAction._ import org.apache.spark.util.Utils /** @@ -39,8 +40,6 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St var driverExtraClassPath: String = null var driverExtraLibraryPath: String = null var driverExtraJavaOptions: String = null - var driverCores: String = null - var supervise: Boolean = false var queue: String = null var numExecutors: String = null var files: String = null @@ -56,8 +55,16 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St var verbose: Boolean = false var isPython: Boolean = false var pyFiles: String = null + var action: SparkSubmitAction = null val sparkProperties: HashMap[String, String] = new HashMap[String, String]() + // Standalone cluster mode only + var supervise: Boolean = false + var driverCores: String = null + var submissionToKill: String = null + var submissionToRequestStatusFor: String = null + var useRest: Boolean = true // used internally + /** Default properties present in the currently defined defaults file. */ lazy val defaultSparkProperties: HashMap[String, String] = { val defaultProperties = new HashMap[String, String]() @@ -82,7 +89,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St // Use `sparkProperties` map along with env vars to fill in any missing parameters loadEnvironmentArguments() - checkRequiredArguments() + validateArguments() /** * Merge values from the default properties file with those specified through --conf. @@ -107,6 +114,15 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St .orElse(sparkProperties.get("spark.master")) .orElse(env.get("MASTER")) .orNull + driverExtraClassPath = Option(driverExtraClassPath) + .orElse(sparkProperties.get("spark.driver.extraClassPath")) + .orNull + driverExtraJavaOptions = Option(driverExtraJavaOptions) + .orElse(sparkProperties.get("spark.driver.extraJavaOptions")) + .orNull + driverExtraLibraryPath = Option(driverExtraLibraryPath) + .orElse(sparkProperties.get("spark.driver.extraLibraryPath")) + .orNull driverMemory = Option(driverMemory) .orElse(sparkProperties.get("spark.driver.memory")) .orElse(env.get("SPARK_DRIVER_MEMORY")) @@ -166,10 +182,21 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St if (name == null && primaryResource != null) { name = Utils.stripDirectory(primaryResource) } + + // Action should be SUBMIT unless otherwise specified + action = Option(action).getOrElse(SUBMIT) } /** Ensure that required fields exists. Call this only once all defaults are loaded. */ - private def checkRequiredArguments(): Unit = { + private def validateArguments(): Unit = { + action match { + case SUBMIT => validateSubmitArguments() + case KILL => validateKillArguments() + case REQUEST_STATUS => validateStatusRequestArguments() + } + } + + private def validateSubmitArguments(): Unit = { if (args.length == 0) { printUsageAndExit(-1) } @@ -192,6 +219,29 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St } } + private def validateKillArguments(): Unit = { + if (!master.startsWith("spark://")) { + SparkSubmit.printErrorAndExit("Killing submissions is only supported in standalone mode!") + } + if (submissionToKill == null) { + SparkSubmit.printErrorAndExit("Please specify a submission to kill.") + } + } + + private def validateStatusRequestArguments(): Unit = { + if (!master.startsWith("spark://")) { + SparkSubmit.printErrorAndExit( + "Requesting submission statuses is only supported in standalone mode!") + } + if (submissionToRequestStatusFor == null) { + SparkSubmit.printErrorAndExit("Please specify a submission to request status for.") + } + } + + def isStandaloneCluster: Boolean = { + master.startsWith("spark://") && deployMode == "cluster" + } + override def toString = { s"""Parsed arguments: | master $master @@ -300,6 +350,22 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St propertiesFile = value parse(tail) + case ("--kill") :: value :: tail => + submissionToKill = value + if (action != null) { + SparkSubmit.printErrorAndExit(s"Action cannot be both $action and $KILL.") + } + action = KILL + parse(tail) + + case ("--status") :: value :: tail => + submissionToRequestStatusFor = value + if (action != null) { + SparkSubmit.printErrorAndExit(s"Action cannot be both $action and $REQUEST_STATUS.") + } + action = REQUEST_STATUS + parse(tail) + case ("--supervise") :: tail => supervise = true parse(tail) @@ -372,7 +438,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St outStream.println("Unknown/unsupported param " + unknownParam) } outStream.println( - """Usage: spark-submit [options] [app options] + """Usage: spark-submit [options] [app arguments] + |Usage: spark-submit --kill [submission ID] --master [spark://...] + |Usage: spark-submit --status [submission ID] --master [spark://...] + | |Options: | --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local. | --deploy-mode DEPLOY_MODE Whether to launch the driver program locally ("client") or @@ -413,6 +482,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St | Spark standalone with cluster deploy mode only: | --driver-cores NUM Cores for driver (Default: 1). | --supervise If given, restarts the driver on failure. + | --kill SUBMISSION_ID If given, kills the driver specified. + | --status SUBMISSION_ID If given, requests the status of the driver specified. | | Spark standalone and Mesos only: | --total-executor-cores NUM Total cores for all executors. 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 5eeb9fe526248..b8b1a25abff2e 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 @@ -43,6 +43,7 @@ import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI +import org.apache.spark.deploy.rest.StandaloneRestServer import org.apache.spark.metrics.MetricsSystem import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} import org.apache.spark.ui.SparkUI @@ -52,12 +53,12 @@ private[spark] class Master( host: String, port: Int, webUiPort: Int, - val securityMgr: SecurityManager) + val securityMgr: SecurityManager, + val conf: SparkConf) extends Actor with ActorLogReceive with Logging with LeaderElectable { import context.dispatcher // to use Akka's scheduler.schedule() - val conf = new SparkConf val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs @@ -121,6 +122,17 @@ private[spark] class Master( throw new SparkException("spark.deploy.defaultCores must be positive") } + // Alternative application submission gateway that is stable across Spark versions + private val restServerEnabled = conf.getBoolean("spark.master.rest.enabled", true) + private val restServer = + if (restServerEnabled) { + val port = conf.getInt("spark.master.rest.port", 6066) + Some(new StandaloneRestServer(host, port, self, masterUrl, conf)) + } else { + None + } + private val restServerBoundPort = restServer.map(_.start()) + override def preStart() { logInfo("Starting Spark master at " + masterUrl) logInfo(s"Running Spark version ${org.apache.spark.SPARK_VERSION}") @@ -174,6 +186,7 @@ private[spark] class Master( recoveryCompletionTask.cancel() } webUi.stop() + restServer.foreach(_.stop()) masterMetricsSystem.stop() applicationMetricsSystem.stop() persistenceEngine.close() @@ -421,7 +434,9 @@ private[spark] class Master( } case RequestMasterState => { - sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray, + sender ! MasterStateResponse( + host, port, restServerBoundPort, + workers.toArray, apps.toArray, completedApps.toArray, drivers.toArray, completedDrivers.toArray, state) } @@ -429,8 +444,8 @@ private[spark] class Master( timeOutDeadWorkers() } - case RequestWebUIPort => { - sender ! WebUIPortResponse(webUi.boundPort) + case BoundPortsRequest => { + sender ! BoundPortsResponse(port, webUi.boundPort, restServerBoundPort) } } @@ -851,7 +866,7 @@ private[spark] object Master extends Logging { SignalLogger.register(log) val conf = new SparkConf val args = new MasterArguments(argStrings, conf) - val (actorSystem, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort, conf) + val (actorSystem, _, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort, conf) actorSystem.awaitTermination() } @@ -875,19 +890,26 @@ private[spark] object Master extends Logging { Address(protocol, systemName, host, port) } + /** + * Start the Master and return a four tuple of: + * (1) The Master actor system + * (2) The bound port + * (3) The web UI bound port + * (4) The REST server bound port, if any + */ def startSystemAndActor( host: String, port: Int, webUiPort: Int, - conf: SparkConf): (ActorSystem, Int, Int) = { + conf: SparkConf): (ActorSystem, Int, Int, Option[Int]) = { val securityMgr = new SecurityManager(conf) val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, conf = conf, securityManager = securityMgr) - val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort, - securityMgr), actorName) + val actor = actorSystem.actorOf( + Props(classOf[Master], host, boundPort, webUiPort, securityMgr, conf), actorName) val timeout = AkkaUtils.askTimeout(conf) - val respFuture = actor.ask(RequestWebUIPort)(timeout) - val resp = Await.result(respFuture, timeout).asInstanceOf[WebUIPortResponse] - (actorSystem, boundPort, resp.webUIBoundPort) + val portsRequest = actor.ask(BoundPortsRequest)(timeout) + val portsResponse = Await.result(portsRequest, timeout).asInstanceOf[BoundPortsResponse] + (actorSystem, boundPort, portsResponse.webUIPort, portsResponse.restPort) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala index db72d8ae9bdaf..15c6296888f70 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala @@ -36,7 +36,7 @@ private[master] object MasterMessages { case object CompleteRecovery - case object RequestWebUIPort + case object BoundPortsRequest - case class WebUIPortResponse(webUIBoundPort: Int) + case class BoundPortsResponse(actorPort: Int, webUIPort: Int, restPort: Option[Int]) } 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 7ca3b08a28728..b47a081053e77 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 @@ -46,19 +46,19 @@ private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, timeout) - val workerHeaders = Seq("Id", "Address", "State", "Cores", "Memory") + val workerHeaders = Seq("Worker Id", "Address", "State", "Cores", "Memory") val workers = state.workers.sortBy(_.id) val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers) - val appHeaders = Seq("ID", "Name", "Cores", "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(appHeaders, appRow, activeApps) val completedApps = state.completedApps.sortBy(_.endTime).reverse val completedAppsTable = UIUtils.listingTable(appHeaders, appRow, completedApps) - val driverHeaders = Seq("ID", "Submitted Time", "Worker", "State", "Cores", "Memory", - "Main Class") + val driverHeaders = Seq("Submission ID", "Submitted Time", "Worker", "State", "Cores", + "Memory", "Main Class") val activeDrivers = state.activeDrivers.sortBy(_.startTime).reverse val activeDriversTable = UIUtils.listingTable(driverHeaders, driverRow, activeDrivers) val completedDrivers = state.completedDrivers.sortBy(_.startTime).reverse @@ -73,6 +73,14 @@ private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
    • URL: {state.uri}
    • + { + state.restUri.map { uri => +
    • + REST URL: {uri} + (cluster mode) +
    • + }.getOrElse { Seq.empty } + }
    • Workers: {state.workers.size}
    • Cores: {state.workers.map(_.cores).sum} Total, {state.workers.map(_.coresUsed).sum} Used
    • 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 new file mode 100644 index 0000000000000..115aa5278bb62 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestClient.scala @@ -0,0 +1,307 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.rest + +import java.io.{DataOutputStream, FileNotFoundException} +import java.net.{HttpURLConnection, SocketException, URL} + +import scala.io.Source + +import com.fasterxml.jackson.databind.JsonMappingException +import com.google.common.base.Charsets + +import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} + +/** + * A client that submits applications to the standalone Master using a REST protocol. + * This client is intended to communicate with the [[StandaloneRestServer]] and is + * currently used for cluster mode only. + * + * In protocol version v1, the REST URL takes the form http://[host:port]/v1/submissions/[action], + * where [action] can be one of create, kill, or status. Each type of request is represented in + * an HTTP message sent to the following prefixes: + * (1) submit - POST to /submissions/create + * (2) kill - POST /submissions/kill/[submissionId] + * (3) status - GET /submissions/status/[submissionId] + * + * In the case of (1), parameters are posted in the HTTP body in the form of JSON fields. + * Otherwise, the URL fully specifies the intended action of the client. + * + * Since the protocol is expected to be stable across Spark versions, existing fields cannot be + * added or removed, though new optional fields can be added. In the rare event that forward or + * backward compatibility is broken, Spark must introduce a new protocol version (e.g. v2). + * + * The client and the server must communicate using the same version of the protocol. If there + * is a mismatch, the server will respond with the highest protocol version it supports. A future + * implementation of this client can use that information to retry using the version specified + * by the server. + */ +private[spark] class StandaloneRestClient extends Logging { + import StandaloneRestClient._ + + /** + * Submit an application specified by the parameters in the provided request. + * + * 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( + master: String, + request: CreateSubmissionRequest): SubmitRestProtocolResponse = { + logInfo(s"Submitting a request to launch an application in $master.") + validateMaster(master) + val url = getSubmitUrl(master) + val response = postJson(url, request.toJson) + response match { + case s: CreateSubmissionResponse => + reportSubmissionStatus(master, s) + handleRestResponse(s) + case unexpected => + handleUnexpectedRestResponse(unexpected) + } + response + } + + /** Request that the server kill the specified submission. */ + def killSubmission(master: String, submissionId: String): SubmitRestProtocolResponse = { + logInfo(s"Submitting a request to kill submission $submissionId in $master.") + validateMaster(master) + val response = post(getKillUrl(master, submissionId)) + response match { + case k: KillSubmissionResponse => handleRestResponse(k) + case unexpected => handleUnexpectedRestResponse(unexpected) + } + response + } + + /** Request the status of a submission from the server. */ + def requestSubmissionStatus( + master: String, + submissionId: String, + quiet: Boolean = false): SubmitRestProtocolResponse = { + logInfo(s"Submitting a request for the status of submission $submissionId in $master.") + validateMaster(master) + val response = get(getStatusUrl(master, submissionId)) + response match { + case s: SubmissionStatusResponse => if (!quiet) { handleRestResponse(s) } + case unexpected => handleUnexpectedRestResponse(unexpected) + } + response + } + + /** Construct a message that captures the specified parameters for submitting an application. */ + def constructSubmitRequest( + appResource: String, + mainClass: String, + appArgs: Array[String], + sparkProperties: Map[String, String], + environmentVariables: Map[String, String]): CreateSubmissionRequest = { + val message = new CreateSubmissionRequest + message.clientSparkVersion = sparkVersion + message.appResource = appResource + message.mainClass = mainClass + message.appArgs = appArgs + message.sparkProperties = sparkProperties + message.environmentVariables = environmentVariables + message.validate() + message + } + + /** Send a GET request to the specified URL. */ + private def get(url: URL): SubmitRestProtocolResponse = { + logDebug(s"Sending GET request to server at $url.") + val conn = url.openConnection().asInstanceOf[HttpURLConnection] + conn.setRequestMethod("GET") + readResponse(conn) + } + + /** Send a POST request to the specified URL. */ + private def post(url: URL): SubmitRestProtocolResponse = { + logDebug(s"Sending POST request to server at $url.") + val conn = url.openConnection().asInstanceOf[HttpURLConnection] + conn.setRequestMethod("POST") + readResponse(conn) + } + + /** Send a POST request with the given JSON as the body to the specified URL. */ + private def postJson(url: URL, json: String): SubmitRestProtocolResponse = { + logDebug(s"Sending POST request to server at $url:\n$json") + val conn = url.openConnection().asInstanceOf[HttpURLConnection] + conn.setRequestMethod("POST") + conn.setRequestProperty("Content-Type", "application/json") + conn.setRequestProperty("charset", "utf-8") + conn.setDoOutput(true) + val out = new DataOutputStream(conn.getOutputStream) + out.write(json.getBytes(Charsets.UTF_8)) + out.close() + readResponse(conn) + } + + /** + * 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. + */ + private def readResponse(connection: HttpURLConnection): SubmitRestProtocolResponse = { + try { + val responseJson = Source.fromInputStream(connection.getInputStream).mkString + logDebug(s"Response from the server:\n$responseJson") + val response = SubmitRestProtocolMessage.fromJson(responseJson) + response.validate() + response match { + // If the response is an error, log the message + case error: ErrorResponse => + logError(s"Server responded with error:\n${error.message}") + error + // Otherwise, simply return the response + case response: SubmitRestProtocolResponse => response + case unexpected => + throw new SubmitRestProtocolException( + s"Message received from server was not a response:\n${unexpected.toJson}") + } + } catch { + case unreachable @ (_: FileNotFoundException | _: SocketException) => + throw new SubmitRestConnectionException( + s"Unable to connect to server ${connection.getURL}", unreachable) + case malformed @ (_: SubmitRestProtocolException | _: JsonMappingException) => + throw new SubmitRestProtocolException( + "Malformed response received from server", malformed) + } + } + + /** Return the REST URL for creating a new submission. */ + private def getSubmitUrl(master: String): URL = { + val baseUrl = getBaseUrl(master) + new URL(s"$baseUrl/create") + } + + /** Return the REST URL for killing an existing submission. */ + private def getKillUrl(master: String, submissionId: String): URL = { + val baseUrl = getBaseUrl(master) + new URL(s"$baseUrl/kill/$submissionId") + } + + /** Return the REST URL for requesting the status of an existing submission. */ + private def getStatusUrl(master: String, submissionId: String): URL = { + val baseUrl = getBaseUrl(master) + new URL(s"$baseUrl/status/$submissionId") + } + + /** Return the base URL for communicating with the server, including the protocol version. */ + private def getBaseUrl(master: String): String = { + val masterUrl = master.stripPrefix("spark://").stripSuffix("/") + s"http://$masterUrl/$PROTOCOL_VERSION/submissions" + } + + /** Throw an exception if this is not standalone mode. */ + private def validateMaster(master: String): Unit = { + if (!master.startsWith("spark://")) { + throw new IllegalArgumentException("This REST client is only supported in standalone mode.") + } + } + + /** Report the status of a newly created submission. */ + private def reportSubmissionStatus( + master: String, + submitResponse: CreateSubmissionResponse): Unit = { + if (submitResponse.success) { + val submissionId = submitResponse.submissionId + if (submissionId != null) { + logInfo(s"Submission successfully created as $submissionId. Polling submission state...") + pollSubmissionStatus(master, submissionId) + } else { + // should never happen + logError("Application successfully submitted, but submission ID was not provided!") + } + } else { + val failMessage = Option(submitResponse.message).map { ": " + _ }.getOrElse("") + logError("Application submission failed" + failMessage) + } + } + + /** + * Poll the status of the specified submission and log it. + * This retries up to a fixed number of times before giving up. + */ + private def pollSubmissionStatus(master: String, submissionId: String): Unit = { + (1 to REPORT_DRIVER_STATUS_MAX_TRIES).foreach { _ => + val response = requestSubmissionStatus(master, submissionId, quiet = true) + val statusResponse = response match { + case s: SubmissionStatusResponse => s + case _ => return // unexpected type, let upstream caller handle it + } + if (statusResponse.success) { + val driverState = Option(statusResponse.driverState) + val workerId = Option(statusResponse.workerId) + val workerHostPort = Option(statusResponse.workerHostPort) + val exception = Option(statusResponse.message) + // Log driver state, if present + driverState match { + case Some(state) => logInfo(s"State of driver $submissionId is now $state.") + case _ => logError(s"State of driver $submissionId was not found!") + } + // Log worker node, if present + (workerId, workerHostPort) match { + case (Some(id), Some(hp)) => logInfo(s"Driver is running on worker $id at $hp.") + case _ => + } + // Log exception stack trace, if present + exception.foreach { e => logError(e) } + return + } + Thread.sleep(REPORT_DRIVER_STATUS_INTERVAL) + } + logError(s"Error: Master did not recognize driver $submissionId.") + } + + /** Log the response sent by the server in the REST application submission protocol. */ + private def handleRestResponse(response: SubmitRestProtocolResponse): Unit = { + logInfo(s"Server responded with ${response.messageType}:\n${response.toJson}") + } + + /** Log an appropriate error if the response sent by the server is not of the expected type. */ + private def handleUnexpectedRestResponse(unexpected: SubmitRestProtocolResponse): Unit = { + logError(s"Error: Server responded with message of unexpected type ${unexpected.messageType}.") + } +} + +private[spark] object StandaloneRestClient { + val REPORT_DRIVER_STATUS_INTERVAL = 1000 + val REPORT_DRIVER_STATUS_MAX_TRIES = 10 + val PROTOCOL_VERSION = "v1" + + /** Submit an application, assuming Spark parameters are specified through system properties. */ + def main(args: Array[String]): Unit = { + if (args.size < 2) { + sys.error("Usage: StandaloneRestClient [app resource] [main class] [app args*]") + sys.exit(1) + } + val appResource = args(0) + 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) + } +} 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 new file mode 100644 index 0000000000000..2033d67e1f394 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -0,0 +1,449 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.rest + +import java.io.{DataOutputStream, 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 org.eclipse.jetty.server.Server +import org.eclipse.jetty.servlet.{ServletHolder, ServletContextHandler} +import org.eclipse.jetty.util.thread.QueuedThreadPool +import org.json4s._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.{Logging, SparkConf, SPARK_VERSION => sparkVersion} +import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.deploy.{Command, DeployMessages, DriverDescription} +import org.apache.spark.deploy.ClientArguments._ + +/** + * A server that responds to requests submitted by the [[StandaloneRestClient]]. + * This is intended to be embedded in the standalone Master and used in cluster mode only. + * + * This server responds with different HTTP codes depending on the situation: + * 200 OK - Request was processed successfully + * 400 BAD REQUEST - Request was malformed, not successfully validated, or of unexpected type + * 468 UNKNOWN PROTOCOL VERSION - Request specified a protocol this server does not understand + * 500 INTERNAL SERVER ERROR - Server throws an exception internally while processing the request + * + * The server always includes a JSON representation of the relevant [[SubmitRestProtocolResponse]] + * in the HTTP body. If an error occurs, however, the server will include an [[ErrorResponse]] + * instead of the one expected by the client. If the construction of this error response itself + * fails, the response will consist of an empty body with a response code that indicates internal + * server error. + * + * @param host the address this server should bind to + * @param requestedPort the port this server will attempt to bind to + * @param masterActor reference to the Master actor to which requests can be sent + * @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( + host: String, + requestedPort: Int, + masterActor: ActorRef, + masterUrl: String, + masterConf: SparkConf) + extends Logging { + + 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 + ) + + /** Start the server and return the bound port. */ + def start(): Int = { + val (server, boundPort) = Utils.startServiceOnPort[Server](requestedPort, doStart, masterConf) + _server = Some(server) + logInfo(s"Started REST server for submitting applications on port $boundPort") + boundPort + } + + /** + * Map the servlets to their corresponding contexts and attach them to a server. + * Return a 2-tuple of the started server and the bound port. + */ + private def doStart(startPort: Int): (Server, Int) = { + val server = new Server(new InetSocketAddress(host, startPort)) + val threadPool = new QueuedThreadPool + threadPool.setDaemon(true) + server.setThreadPool(threadPool) + val mainHandler = new ServletContextHandler + mainHandler.setContextPath("/") + servletToContext.foreach { case (servlet, prefix) => + mainHandler.addServlet(new ServletHolder(servlet), prefix) + } + server.setHandler(mainHandler) + server.start() + val boundPort = server.getConnectors()(0).getLocalPort + (server, boundPort) + } + + def stop(): Unit = { + _server.foreach(_.stop()) + } +} + +private object StandaloneRestServer { + val PROTOCOL_VERSION = StandaloneRestClient.PROTOCOL_VERSION + val SC_UNKNOWN_PROTOCOL_VERSION = 468 +} + +/** + * 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) + } + } + + /** + * Serialize the given response message to JSON and send it through the response servlet. + * This validates the response before sending it to ensure it is properly constructed. + */ + protected def sendResponse( + responseMessage: SubmitRestProtocolResponse, + responseServlet: HttpServletResponse): Unit = { + 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() + } + + /** + * Return any fields in the client request message that the server does not know about. + * + * The mechanism for this is to reconstruct the JSON on the server side and compare the + * diff between this JSON and the one generated on the client side. Any fields that are + * only in the client JSON are treated as unexpected. + */ + protected def findUnknownFields( + requestJson: String, + requestMessage: SubmitRestProtocolMessage): Array[String] = { + val clientSideJson = parse(requestJson) + val serverSideJson = parse(requestMessage.toJson) + val Diff(_, _, unknown) = clientSideJson.diff(serverSideJson) + unknown match { + case j: JObject => j.obj.map { case (k, _) => k }.toArray + case _ => Array.empty[String] // No difference + } + } + + /** Return a human readable String representation of the exception. */ + protected def formatException(e: Throwable): String = { + val stackTraceString = e.getStackTrace.map { "\t" + _ }.mkString("\n") + s"$e\n$stackTraceString" + } + + /** Construct an error message to signal the fact that an exception has been thrown. */ + protected def handleError(message: String): ErrorResponse = { + val e = new ErrorResponse + e.serverSparkVersion = sparkVersion + e.message = message + e + } + + /** + * Validate the response to ensure that it is correctly constructed. + * + * If it is, simply return the message as is. Otherwise, return an error response instead + * to propagate the exception back to the client and set the appropriate error code. + */ + private def validateResponse( + responseMessage: SubmitRestProtocolResponse, + responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { + try { + responseMessage.validate() + responseMessage + } catch { + case e: Exception => + responseServlet.setStatus(HttpServletResponse.SC_INTERNAL_SERVER_ERROR) + handleError("Internal server error: " + formatException(e)) + } + } +} + +/** + * A servlet for handling kill requests passed to the [[StandaloneRestServer]]. + */ +private class KillRequestServlet(masterActor: ActorRef, conf: SparkConf) + extends StandaloneRestServlet { + + /** + * If a submission ID is specified in the URL, have the Master kill the corresponding + * driver and return an appropriate response to the client. Otherwise, return error. + */ + 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.") + } + sendResponse(responseMessage, response) + } + + private def handleKill(submissionId: String): KillSubmissionResponse = { + val askTimeout = AkkaUtils.askTimeout(conf) + val response = AkkaUtils.askWithReply[DeployMessages.KillDriverResponse]( + DeployMessages.RequestKillDriver(submissionId), masterActor, askTimeout) + val k = new KillSubmissionResponse + k.serverSparkVersion = sparkVersion + k.message = response.message + k.submissionId = submissionId + k.success = response.success + k + } +} + +/** + * A servlet for handling status requests passed to the [[StandaloneRestServer]]. + */ +private class StatusRequestServlet(masterActor: ActorRef, conf: SparkConf) + extends StandaloneRestServlet { + + /** + * If a submission ID is specified in the URL, request the status of the corresponding + * driver from the Master and include it in the response. Otherwise, return error. + */ + 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.") + } + sendResponse(responseMessage, response) + } + + private def handleStatus(submissionId: String): SubmissionStatusResponse = { + val askTimeout = AkkaUtils.askTimeout(conf) + val response = AkkaUtils.askWithReply[DeployMessages.DriverStatusResponse]( + DeployMessages.RequestDriverStatus(submissionId), masterActor, askTimeout) + val message = response.exception.map { s"Exception from the cluster:\n" + formatException(_) } + val d = new SubmissionStatusResponse + d.serverSparkVersion = sparkVersion + d.submissionId = submissionId + d.success = response.found + d.driverState = response.state.map(_.toString).orNull + d.workerId = response.workerId.orNull + d.workerHostPort = response.workerHostPort.orNull + d.message = message.orNull + d + } +} + +/** + * A servlet for handling submit requests passed to the [[StandaloneRestServer]]. + */ +private class SubmitRequestServlet( + masterActor: ActorRef, + masterUrl: String, + conf: SparkConf) + extends StandaloneRestServlet { + + /** + * Submit an application to the Master with parameters specified in the request. + * + * The request is assumed to be a [[SubmitRestProtocolRequest]] in the form of JSON. + * If the request is successfully processed, return an appropriate response to the + * client indicating so. Otherwise, return error instead. + */ + protected override def doPost( + requestServlet: HttpServletRequest, + responseServlet: HttpServletResponse): Unit = { + val responseMessage = + try { + val requestMessageJson = Source.fromInputStream(requestServlet.getInputStream).mkString + val requestMessage = SubmitRestProtocolMessage.fromJson(requestMessageJson) + // The response should have already been validated on the client. + // In case this is not true, validate it ourselves to avoid potential NPEs. + requestMessage.validate() + handleSubmit(requestMessageJson, requestMessage, responseServlet) + } catch { + // The client failed to provide a valid JSON, so this is not our fault + case e @ (_: JsonMappingException | _: SubmitRestProtocolException) => + responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) + handleError("Malformed request: " + formatException(e)) + } + sendResponse(responseMessage, responseServlet) + } + + /** + * Handle the submit request and construct an appropriate response to return to the client. + * + * This assumes that the request message is already successfully validated. + * If the request message is not of the expected type, return error to the client. + */ + private def handleSubmit( + requestMessageJson: String, + requestMessage: SubmitRestProtocolMessage, + responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { + requestMessage match { + case submitRequest: CreateSubmissionRequest => + val askTimeout = AkkaUtils.askTimeout(conf) + val driverDescription = buildDriverDescription(submitRequest) + val response = AkkaUtils.askWithReply[DeployMessages.SubmitDriverResponse]( + DeployMessages.RequestSubmitDriver(driverDescription), masterActor, askTimeout) + val submitResponse = new CreateSubmissionResponse + submitResponse.serverSparkVersion = sparkVersion + submitResponse.message = response.message + submitResponse.success = response.success + submitResponse.submissionId = response.driverId.orNull + val unknownFields = findUnknownFields(requestMessageJson, requestMessage) + if (unknownFields.nonEmpty) { + // If there are fields that the server does not know about, warn the client + submitResponse.unknownFields = unknownFields + } + submitResponse + case unexpected => + responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) + handleError(s"Received message of unexpected type ${unexpected.messageType}.") + } + } + + /** + * Build a driver description from the fields specified in the submit request. + * + * This involves constructing a command that takes into account memory, java options, + * classpath and other settings to launch the driver. This does not currently consider + * fields used by python applications since python is not supported in standalone + * cluster mode yet. + */ + private def buildDriverDescription(request: CreateSubmissionRequest): DriverDescription = { + // Required fields, including the main class because python is not yet supported + val appResource = Option(request.appResource).getOrElse { + throw new SubmitRestMissingFieldException("Application jar is missing.") + } + val mainClass = Option(request.mainClass).getOrElse { + throw new SubmitRestMissingFieldException("Main class is missing.") + } + + // Optional fields + val sparkProperties = request.sparkProperties + val driverMemory = sparkProperties.get("spark.driver.memory") + val driverCores = sparkProperties.get("spark.driver.cores") + val driverExtraJavaOptions = sparkProperties.get("spark.driver.extraJavaOptions") + val driverExtraClassPath = sparkProperties.get("spark.driver.extraClassPath") + val driverExtraLibraryPath = sparkProperties.get("spark.driver.extraLibraryPath") + val superviseDriver = sparkProperties.get("spark.driver.supervise") + val appArgs = request.appArgs + val environmentVariables = request.environmentVariables + + // Construct driver description + val conf = new SparkConf(false) + .setAll(sparkProperties) + .set("spark.master", masterUrl) + val extraClassPath = driverExtraClassPath.toSeq.flatMap(_.split(File.pathSeparator)) + val extraLibraryPath = driverExtraLibraryPath.toSeq.flatMap(_.split(File.pathSeparator)) + val extraJavaOpts = driverExtraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) + val sparkJavaOpts = Utils.sparkJavaOpts(conf) + val javaOpts = sparkJavaOpts ++ extraJavaOpts + val command = new Command( + "org.apache.spark.deploy.worker.DriverWrapper", + Seq("{{WORKER_URL}}", 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) + val actualSuperviseDriver = superviseDriver.map(_.toBoolean).getOrElse(DEFAULT_SUPERVISE) + new DriverDescription( + appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, command) + } +} + +/** + * A default servlet that handles error cases that are not captured by other servlets. + */ +private class ErrorServlet extends StandaloneRestServlet { + private val serverVersion = StandaloneRestServer.PROTOCOL_VERSION + + /** Service a faulty request by returning an appropriate error message to the client. */ + protected override def service( + request: HttpServletRequest, + response: HttpServletResponse): Unit = { + val path = request.getPathInfo + val parts = path.stripPrefix("/").split("/").toSeq + var versionMismatch = false + var msg = + parts match { + case Nil => + // http://host:port/ + "Missing protocol version." + case `serverVersion` :: Nil => + // http://host:port/correct-version + "Missing the /submissions prefix." + case `serverVersion` :: "submissions" :: Nil => + // http://host:port/correct-version/submissions + "Missing an action: please specify one of /create, /kill, or /status." + case unknownVersion :: _ => + // http://host:port/unknown-version/* + versionMismatch = true + s"Unknown protocol version '$unknownVersion'." + case _ => + // never reached + s"Malformed path $path." + } + msg += s" Please submit requests through http://[host]:[port]/$serverVersion/submissions/..." + val error = handleError(msg) + // If there is a version mismatch, include the highest protocol version that + // this server supports in case the client wants to retry with our version + if (versionMismatch) { + error.highestProtocolVersion = serverVersion + response.setStatus(StandaloneRestServer.SC_UNKNOWN_PROTOCOL_VERSION) + } else { + response.setStatus(HttpServletResponse.SC_BAD_REQUEST) + } + sendResponse(error, response) + } +} 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 new file mode 100644 index 0000000000000..d7a0bdbe10778 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolException.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.deploy.rest + +/** + * An exception thrown in the REST application submission protocol. + */ +private[spark] 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) + extends SubmitRestProtocolException(message) + +/** + * An exception thrown if the REST client cannot reach the REST server. + */ +private[spark] 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 new file mode 100644 index 0000000000000..b877898231e3e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.rest + +import scala.util.Try + +import com.fasterxml.jackson.annotation._ +import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility +import com.fasterxml.jackson.annotation.JsonInclude.Include +import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper, SerializationFeature} +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import org.json4s.JsonAST._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.util.Utils + +/** + * An abstract message exchanged in the REST application submission protocol. + * + * This message is intended to be serialized to and deserialized from JSON in the exchange. + * Each message can either be a request or a response and consists of three common fields: + * (1) the action, which fully specifies the type of the message + * (2) the Spark version of the client / server + * (3) an optional message + */ +@JsonInclude(Include.NON_NULL) +@JsonAutoDetect(getterVisibility = Visibility.ANY, setterVisibility = Visibility.ANY) +@JsonPropertyOrder(alphabetic = true) +private[spark] abstract class SubmitRestProtocolMessage { + @JsonIgnore + val messageType = Utils.getFormattedClassName(this) + + val action: String = messageType + var message: String = null + + // For JSON deserialization + private def setAction(a: String): Unit = { } + + /** + * Serialize the message to JSON. + * This also ensures that the message is valid and its fields are in the expected format. + */ + def toJson: String = { + validate() + SubmitRestProtocolMessage.mapper.writeValueAsString(this) + } + + /** + * Assert the validity of the message. + * If the validation fails, throw a [[SubmitRestProtocolException]]. + */ + final def validate(): Unit = { + try { + doValidate() + } catch { + case e: Exception => + throw new SubmitRestProtocolException(s"Validation of message $messageType failed!", e) + } + } + + /** Assert the validity of the message */ + protected def doValidate(): Unit = { + if (action == null) { + throw new SubmitRestMissingFieldException(s"The action field is missing in $messageType") + } + } + + /** Assert that the specified field is set in this message. */ + protected def assertFieldIsSet[T](value: T, name: String): Unit = { + if (value == null) { + throw new SubmitRestMissingFieldException(s"'$name' is missing in message $messageType.") + } + } + + /** + * Assert a condition when validating this message. + * If the assertion fails, throw a [[SubmitRestProtocolException]]. + */ + protected def assert(condition: Boolean, failMessage: String): Unit = { + if (!condition) { throw new SubmitRestProtocolException(failMessage) } + } +} + +/** + * Helper methods to process serialized [[SubmitRestProtocolMessage]]s. + */ +private[spark] object SubmitRestProtocolMessage { + private val packagePrefix = this.getClass.getPackage.getName + private val mapper = new ObjectMapper() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) + .enable(SerializationFeature.INDENT_OUTPUT) + .registerModule(DefaultScalaModule) + + /** + * Parse the value of the action field from the given JSON. + * 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") + } + } + + /** + * Construct a [[SubmitRestProtocolMessage]] from its JSON representation. + * + * This method first parses the action from the JSON and uses it to infer the message type. + * Note that the action must represent one of the [[SubmitRestProtocolMessage]]s defined in + * this package. Otherwise, a [[ClassNotFoundException]] will be thrown. + */ + def fromJson(json: String): SubmitRestProtocolMessage = { + val className = parseAction(json) + val clazz = Class.forName(packagePrefix + "." + className) + .asSubclass[SubmitRestProtocolMessage](classOf[SubmitRestProtocolMessage]) + fromJson(json, clazz) + } + + /** + * Construct a [[SubmitRestProtocolMessage]] from its JSON representation. + * + * This method determines the type of the message from the class provided instead of + * inferring it from the action field. This is useful for deserializing JSON that + * represents custom user-defined messages. + */ + def fromJson[T <: SubmitRestProtocolMessage](json: String, clazz: Class[T]): T = { + mapper.readValue(json, clazz) + } +} 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 new file mode 100644 index 0000000000000..9e1fd8c40cabd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.rest + +import scala.util.Try + +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 { + var clientSparkVersion: String = null + protected override def doValidate(): Unit = { + super.doValidate() + assertFieldIsSet(clientSparkVersion, "clientSparkVersion") + } +} + +/** + * A request to launch a new application in the REST application submission protocol. + */ +private[spark] class CreateSubmissionRequest extends SubmitRestProtocolRequest { + var appResource: String = null + var mainClass: String = null + var appArgs: Array[String] = null + var sparkProperties: Map[String, String] = null + var environmentVariables: Map[String, String] = null + + protected override def doValidate(): Unit = { + super.doValidate() + assert(sparkProperties != null, "No Spark properties set!") + assertFieldIsSet(appResource, "appResource") + assertPropertyIsSet("spark.app.name") + assertPropertyIsBoolean("spark.driver.supervise") + assertPropertyIsNumeric("spark.driver.cores") + assertPropertyIsNumeric("spark.cores.max") + assertPropertyIsMemory("spark.driver.memory") + assertPropertyIsMemory("spark.executor.memory") + } + + private def assertPropertyIsSet(key: String): Unit = + assertFieldIsSet(sparkProperties.getOrElse(key, null), key) + + private def assertPropertyIsBoolean(key: String): Unit = + assertProperty[Boolean](key, "boolean", _.toBoolean) + + private def assertPropertyIsNumeric(key: String): Unit = + assertProperty[Int](key, "numeric", _.toInt) + + private def assertPropertyIsMemory(key: String): Unit = + assertProperty[Int](key, "memory", Utils.memoryStringToMb) + + /** Assert that a Spark property can be converted to a certain type. */ + private def assertProperty[T](key: String, valueType: String, convert: (String => T)): Unit = { + sparkProperties.get(key).foreach { value => + Try(convert(value)).getOrElse { + throw new SubmitRestProtocolException( + s"Property '$key' expected $valueType value: actual was '$value'.") + } + } + } +} 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 new file mode 100644 index 0000000000000..16dfe041d4bea --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolResponse.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.deploy.rest + +import java.lang.Boolean + +/** + * An abstract response sent from the server in the REST application submission protocol. + */ +private[spark] abstract class SubmitRestProtocolResponse extends SubmitRestProtocolMessage { + var serverSparkVersion: String = null + var success: Boolean = null + var unknownFields: Array[String] = null + protected override def doValidate(): Unit = { + super.doValidate() + assertFieldIsSet(serverSparkVersion, "serverSparkVersion") + } +} + +/** + * A response to a [[CreateSubmissionRequest]] in the REST application submission protocol. + */ +private[spark] class CreateSubmissionResponse extends SubmitRestProtocolResponse { + var submissionId: String = null + protected override def doValidate(): Unit = { + super.doValidate() + assertFieldIsSet(success, "success") + } +} + +/** + * A response to a kill request in the REST application submission protocol. + */ +private[spark] class KillSubmissionResponse extends SubmitRestProtocolResponse { + var submissionId: String = null + protected override def doValidate(): Unit = { + super.doValidate() + assertFieldIsSet(submissionId, "submissionId") + assertFieldIsSet(success, "success") + } +} + +/** + * A response to a status request in the REST application submission protocol. + */ +private[spark] class SubmissionStatusResponse extends SubmitRestProtocolResponse { + var submissionId: String = null + var driverState: String = null + var workerId: String = null + var workerHostPort: String = null + + protected override def doValidate(): Unit = { + super.doValidate() + assertFieldIsSet(submissionId, "submissionId") + assertFieldIsSet(success, "success") + } +} + +/** + * An error response message used in the REST application submission protocol. + */ +private[spark] 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 + protected override def doValidate(): Unit = { + super.doValidate() + assertFieldIsSet(message, "message") + } +} 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 ed02ca81e405c..e955636cf5b59 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -68,7 +68,8 @@ class JsonProtocolSuite extends FunSuite { val completedApps = Array[ApplicationInfo]() val activeDrivers = Array(createDriverInfo()) val completedDrivers = Array(createDriverInfo()) - val stateResponse = new MasterStateResponse("host", 8080, workers, activeApps, completedApps, + val stateResponse = new MasterStateResponse( + "host", 8080, None, workers, activeApps, completedApps, activeDrivers, completedDrivers, RecoveryState.ALIVE) val output = JsonProtocol.writeMasterState(stateResponse) assertValidJson(output) 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 3f1355f82893e..1ddccae1262bc 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -141,7 +141,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) - val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs) val childArgsStr = childArgs.mkString(" ") childArgsStr should include ("--class org.SomeClass") childArgsStr should include ("--executor-memory 5g") @@ -180,7 +180,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) - val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") mainClass should be ("org.SomeClass") classpath should have length (4) @@ -201,6 +201,18 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties } test("handles standalone cluster mode") { + testStandaloneCluster(useRest = true) + } + + test("handles legacy standalone cluster mode") { + testStandaloneCluster(useRest = false) + } + + /** + * Test whether the launch environment is correctly set up in standalone cluster mode. + * @param useRest whether to use the REST submission gateway introduced in Spark 1.3 + */ + private def testStandaloneCluster(useRest: Boolean): Unit = { val clArgs = Seq( "--deploy-mode", "cluster", "--master", "spark://h:p", @@ -212,17 +224,26 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) - val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + appArgs.useRest = useRest + val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs) val childArgsStr = childArgs.mkString(" ") - childArgsStr should startWith ("--memory 4g --cores 5 --supervise") - childArgsStr should include regex ("launch spark://h:p .*thejar.jar org.SomeClass arg1 arg2") - mainClass should be ("org.apache.spark.deploy.Client") - classpath should have size (0) - sysProps should have size (5) + if (useRest) { + childArgsStr should endWith ("thejar.jar org.SomeClass arg1 arg2") + mainClass should be ("org.apache.spark.deploy.rest.StandaloneRestClient") + } else { + childArgsStr should startWith ("--supervise --memory 4g --cores 5") + childArgsStr should include regex "launch spark://h:p .*thejar.jar org.SomeClass arg1 arg2" + mainClass should be ("org.apache.spark.deploy.Client") + } + classpath should have size 0 + sysProps should have size 8 sysProps.keys should contain ("SPARK_SUBMIT") sysProps.keys should contain ("spark.master") sysProps.keys should contain ("spark.app.name") sysProps.keys should contain ("spark.jars") + sysProps.keys should contain ("spark.driver.memory") + sysProps.keys should contain ("spark.driver.cores") + sysProps.keys should contain ("spark.driver.supervise") sysProps.keys should contain ("spark.shuffle.spill") sysProps("spark.shuffle.spill") should be ("false") } @@ -239,7 +260,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) - val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") mainClass should be ("org.SomeClass") classpath should have length (1) @@ -261,7 +282,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) - val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") mainClass should be ("org.SomeClass") classpath should have length (1) @@ -281,7 +302,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) - val (_, _, sysProps, mainClass) = createLaunchEnv(appArgs) + val (_, _, sysProps, mainClass) = prepareSubmitEnvironment(appArgs) sysProps("spark.executor.memory") should be ("5g") sysProps("spark.master") should be ("yarn-cluster") mainClass should be ("org.apache.spark.deploy.yarn.Client") @@ -339,7 +360,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "--files", files, "thejar.jar") val appArgs = new SparkSubmitArguments(clArgs) - val sysProps = SparkSubmit.createLaunchEnv(appArgs)._3 + val sysProps = SparkSubmit.prepareSubmitEnvironment(appArgs)._3 appArgs.jars should be (Utils.resolveURIs(jars)) appArgs.files should be (Utils.resolveURIs(files)) sysProps("spark.jars") should be (Utils.resolveURIs(jars + ",thejar.jar")) @@ -354,7 +375,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "thejar.jar" ) val appArgs2 = new SparkSubmitArguments(clArgs2) - val sysProps2 = SparkSubmit.createLaunchEnv(appArgs2)._3 + val sysProps2 = SparkSubmit.prepareSubmitEnvironment(appArgs2)._3 appArgs2.files should be (Utils.resolveURIs(files)) appArgs2.archives should be (Utils.resolveURIs(archives)) sysProps2("spark.yarn.dist.files") should be (Utils.resolveURIs(files)) @@ -367,7 +388,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "mister.py" ) val appArgs3 = new SparkSubmitArguments(clArgs3) - val sysProps3 = SparkSubmit.createLaunchEnv(appArgs3)._3 + val sysProps3 = SparkSubmit.prepareSubmitEnvironment(appArgs3)._3 appArgs3.pyFiles should be (Utils.resolveURIs(pyFiles)) sysProps3("spark.submit.pyFiles") should be ( PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) @@ -392,7 +413,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "thejar.jar" ) val appArgs = new SparkSubmitArguments(clArgs) - val sysProps = SparkSubmit.createLaunchEnv(appArgs)._3 + val sysProps = SparkSubmit.prepareSubmitEnvironment(appArgs)._3 sysProps("spark.jars") should be(Utils.resolveURIs(jars + ",thejar.jar")) sysProps("spark.files") should be(Utils.resolveURIs(files)) @@ -409,7 +430,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "thejar.jar" ) val appArgs2 = new SparkSubmitArguments(clArgs2) - val sysProps2 = SparkSubmit.createLaunchEnv(appArgs2)._3 + val sysProps2 = SparkSubmit.prepareSubmitEnvironment(appArgs2)._3 sysProps2("spark.yarn.dist.files") should be(Utils.resolveURIs(files)) sysProps2("spark.yarn.dist.archives") should be(Utils.resolveURIs(archives)) @@ -424,7 +445,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties "mister.py" ) val appArgs3 = new SparkSubmitArguments(clArgs3) - val sysProps3 = SparkSubmit.createLaunchEnv(appArgs3)._3 + val sysProps3 = SparkSubmit.prepareSubmitEnvironment(appArgs3)._3 sysProps3("spark.submit.pyFiles") should be( PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) } @@ -440,7 +461,7 @@ class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties val appArgs = new SparkSubmitArguments(args, Map("SPARK_CONF_DIR" -> path)) assert(appArgs.propertiesFile != null) assert(appArgs.propertiesFile.startsWith(path)) - appArgs.executorMemory should be ("2.3g") + appArgs.executorMemory should be ("2.3g") } } 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 new file mode 100644 index 0000000000000..29aed89b67aa7 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -0,0 +1,265 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.rest + +import java.io.{File, FileInputStream, FileOutputStream, PrintWriter} +import java.util.jar.{JarEntry, JarOutputStream} +import java.util.zip.ZipEntry + +import scala.collection.mutable.ArrayBuffer +import scala.io.Source + +import akka.actor.ActorSystem +import com.google.common.io.ByteStreams +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} +import org.scalatest.exceptions.TestFailedException + +import org.apache.spark._ +import org.apache.spark.util.Utils +import org.apache.spark.deploy.{SparkSubmit, SparkSubmitArguments} +import org.apache.spark.deploy.master.{DriverState, Master} +import org.apache.spark.deploy.worker.Worker + +/** + * End-to-end tests for the REST application submission protocol in standalone mode. + */ +class StandaloneRestSubmitSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { + private val systemsToStop = new ArrayBuffer[ActorSystem] + private val masterRestUrl = startLocalCluster() + private val client = new StandaloneRestClient + private val mainJar = StandaloneRestSubmitSuite.createJar() + private val mainClass = StandaloneRestApp.getClass.getName.stripSuffix("$") + + override def afterAll() { + systemsToStop.foreach(_.shutdown()) + } + + 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("kill empty submission") { + val response = client.killSubmission(masterRestUrl, "submission-that-does-not-exist") + val killResponse = getKillResponse(response) + val killSuccess = killResponse.success + assert(!killSuccess) + } + + 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) + 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) } + } + + test("request status for empty submission") { + val response = client.requestSubmissionStatus(masterRestUrl, "submission-that-does-not-exist") + val statusResponse = getStatusResponse(response) + val statusSuccess = statusResponse.success + assert(!statusSuccess) + } + + /** + * 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. + */ + 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 + } + + /** 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) + val commandLineArgs = Array( + "--deploy-mode", "cluster", + "--master", masterRestUrl, + "--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.") + } + } + } + + /** Return the response as a submit response, or fail with error otherwise. */ + private def getSubmitResponse(response: SubmitRestProtocolResponse): CreateSubmissionResponse = { + response match { + case s: CreateSubmissionResponse => s + case e: ErrorResponse => fail(s"Server returned error: ${e.message}") + case r => fail(s"Expected submit response. Actual: ${r.toJson}") + } + } + + /** Return the response as a kill response, or fail with error otherwise. */ + private def getKillResponse(response: SubmitRestProtocolResponse): KillSubmissionResponse = { + response match { + case k: KillSubmissionResponse => k + case e: ErrorResponse => fail(s"Server returned error: ${e.message}") + case r => fail(s"Expected kill response. Actual: ${r.toJson}") + } + } + + /** Return the response as a status response, or fail with error otherwise. */ + private def getStatusResponse(response: SubmitRestProtocolResponse): SubmissionStatusResponse = { + response match { + case s: SubmissionStatusResponse => s + case e: ErrorResponse => fail(s"Server returned error: ${e.message}") + case r => fail(s"Expected status response. Actual: ${r.toJson}") + } + } + + /** 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") + } +} + +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. + */ + 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() + } + jarStream.close() + jarFileStream.close() + jarFile.getAbsolutePath + } + + /** + * Return a list of class files compiled for [[StandaloneRestApp]]. + * This includes all the anonymous classes used in the application. + */ + 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)) + } +} + +/** + * Sample application to be submitted to the cluster using the REST gateway. + * All relevant classes will be packaged into a jar at run time. + */ +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() + } + } +} 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 new file mode 100644 index 0000000000000..1d64ec201e647 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala @@ -0,0 +1,324 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.rest + +import java.lang.Boolean +import java.lang.Integer + +import org.json4s.jackson.JsonMethods._ +import org.scalatest.FunSuite + +import org.apache.spark.SparkConf + +/** + * Tests for the REST application submission protocol. + */ +class SubmitRestProtocolSuite extends FunSuite { + + test("validate") { + val request = new DummyRequest + intercept[SubmitRestProtocolException] { request.validate() } // missing everything + request.clientSparkVersion = "1.2.3" + intercept[SubmitRestProtocolException] { request.validate() } // missing name and age + request.name = "something" + intercept[SubmitRestProtocolException] { request.validate() } // missing only age + request.age = 2 + intercept[SubmitRestProtocolException] { request.validate() } // age too low + request.age = 10 + request.validate() // everything is set properly + request.clientSparkVersion = null + intercept[SubmitRestProtocolException] { request.validate() } // missing only Spark version + request.clientSparkVersion = "1.2.3" + request.name = null + intercept[SubmitRestProtocolException] { request.validate() } // missing only name + request.message = "not-setting-name" + intercept[SubmitRestProtocolException] { request.validate() } // still missing name + } + + test("request to and from JSON") { + val request = new DummyRequest + intercept[SubmitRestProtocolException] { request.toJson } // implicit validation + request.clientSparkVersion = "1.2.3" + request.active = true + request.age = 25 + request.name = "jung" + val json = request.toJson + assertJsonEquals(json, dummyRequestJson) + val newRequest = SubmitRestProtocolMessage.fromJson(json, classOf[DummyRequest]) + assert(newRequest.clientSparkVersion === "1.2.3") + assert(newRequest.clientSparkVersion === "1.2.3") + assert(newRequest.active) + assert(newRequest.age === 25) + assert(newRequest.name === "jung") + assert(newRequest.message === null) + } + + test("response to and from JSON") { + val response = new DummyResponse + response.serverSparkVersion = "3.3.4" + response.success = true + val json = response.toJson + assertJsonEquals(json, dummyResponseJson) + val newResponse = SubmitRestProtocolMessage.fromJson(json, classOf[DummyResponse]) + assert(newResponse.serverSparkVersion === "3.3.4") + assert(newResponse.serverSparkVersion === "3.3.4") + assert(newResponse.success) + assert(newResponse.message === null) + } + + test("CreateSubmissionRequest") { + val message = new CreateSubmissionRequest + intercept[SubmitRestProtocolException] { message.validate() } + message.clientSparkVersion = "1.2.3" + message.appResource = "honey-walnut-cherry.jar" + message.mainClass = "org.apache.spark.examples.SparkPie" + val conf = new SparkConf(false) + conf.set("spark.app.name", "SparkPie") + message.sparkProperties = conf.getAll.toMap + message.validate() + // optional fields + conf.set("spark.jars", "mayonnaise.jar,ketchup.jar") + conf.set("spark.files", "fireball.png") + conf.set("spark.driver.memory", "512m") + conf.set("spark.driver.cores", "180") + conf.set("spark.driver.extraJavaOptions", " -Dslices=5 -Dcolor=mostly_red") + conf.set("spark.driver.extraClassPath", "food-coloring.jar") + conf.set("spark.driver.extraLibraryPath", "pickle.jar") + conf.set("spark.driver.supervise", "false") + conf.set("spark.executor.memory", "256m") + conf.set("spark.cores.max", "10000") + message.sparkProperties = conf.getAll.toMap + message.appArgs = Array("two slices", "a hint of cinnamon") + message.environmentVariables = Map("PATH" -> "/dev/null") + message.validate() + // bad fields + var badConf = conf.clone().set("spark.driver.cores", "one hundred feet") + message.sparkProperties = badConf.getAll.toMap + intercept[SubmitRestProtocolException] { message.validate() } + badConf = conf.clone().set("spark.driver.supervise", "nope, never") + message.sparkProperties = badConf.getAll.toMap + intercept[SubmitRestProtocolException] { message.validate() } + badConf = conf.clone().set("spark.cores.max", "two men") + message.sparkProperties = badConf.getAll.toMap + intercept[SubmitRestProtocolException] { message.validate() } + message.sparkProperties = conf.getAll.toMap + // test JSON + val json = message.toJson + assertJsonEquals(json, submitDriverRequestJson) + val newMessage = SubmitRestProtocolMessage.fromJson(json, classOf[CreateSubmissionRequest]) + assert(newMessage.clientSparkVersion === "1.2.3") + assert(newMessage.appResource === "honey-walnut-cherry.jar") + assert(newMessage.mainClass === "org.apache.spark.examples.SparkPie") + assert(newMessage.sparkProperties("spark.app.name") === "SparkPie") + assert(newMessage.sparkProperties("spark.jars") === "mayonnaise.jar,ketchup.jar") + 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.extraClassPath") === "food-coloring.jar") + assert(newMessage.sparkProperties("spark.driver.extraLibraryPath") === "pickle.jar") + assert(newMessage.sparkProperties("spark.driver.supervise") === "false") + assert(newMessage.sparkProperties("spark.executor.memory") === "256m") + assert(newMessage.sparkProperties("spark.cores.max") === "10000") + assert(newMessage.appArgs === message.appArgs) + assert(newMessage.sparkProperties === message.sparkProperties) + assert(newMessage.environmentVariables === message.environmentVariables) + } + + test("CreateSubmissionResponse") { + val message = new CreateSubmissionResponse + intercept[SubmitRestProtocolException] { message.validate() } + message.serverSparkVersion = "1.2.3" + message.submissionId = "driver_123" + message.success = true + message.validate() + // test JSON + val json = message.toJson + assertJsonEquals(json, submitDriverResponseJson) + val newMessage = SubmitRestProtocolMessage.fromJson(json, classOf[CreateSubmissionResponse]) + assert(newMessage.serverSparkVersion === "1.2.3") + assert(newMessage.submissionId === "driver_123") + assert(newMessage.success) + } + + test("KillSubmissionResponse") { + val message = new KillSubmissionResponse + intercept[SubmitRestProtocolException] { message.validate() } + message.serverSparkVersion = "1.2.3" + message.submissionId = "driver_123" + message.success = true + message.validate() + // test JSON + val json = message.toJson + assertJsonEquals(json, killDriverResponseJson) + val newMessage = SubmitRestProtocolMessage.fromJson(json, classOf[KillSubmissionResponse]) + assert(newMessage.serverSparkVersion === "1.2.3") + assert(newMessage.submissionId === "driver_123") + assert(newMessage.success) + } + + test("SubmissionStatusResponse") { + val message = new SubmissionStatusResponse + intercept[SubmitRestProtocolException] { message.validate() } + message.serverSparkVersion = "1.2.3" + message.submissionId = "driver_123" + message.success = true + message.validate() + // optional fields + message.driverState = "RUNNING" + message.workerId = "worker_123" + message.workerHostPort = "1.2.3.4:7780" + // test JSON + val json = message.toJson + assertJsonEquals(json, driverStatusResponseJson) + val newMessage = SubmitRestProtocolMessage.fromJson(json, classOf[SubmissionStatusResponse]) + assert(newMessage.serverSparkVersion === "1.2.3") + assert(newMessage.submissionId === "driver_123") + assert(newMessage.driverState === "RUNNING") + assert(newMessage.success) + assert(newMessage.workerId === "worker_123") + assert(newMessage.workerHostPort === "1.2.3.4:7780") + } + + test("ErrorResponse") { + val message = new ErrorResponse + intercept[SubmitRestProtocolException] { message.validate() } + message.serverSparkVersion = "1.2.3" + message.message = "Field not found in submit request: X" + message.validate() + // test JSON + val json = message.toJson + assertJsonEquals(json, errorJson) + val newMessage = SubmitRestProtocolMessage.fromJson(json, classOf[ErrorResponse]) + assert(newMessage.serverSparkVersion === "1.2.3") + assert(newMessage.message === "Field not found in submit request: X") + } + + private val dummyRequestJson = + """ + |{ + | "action" : "DummyRequest", + | "active" : true, + | "age" : 25, + | "clientSparkVersion" : "1.2.3", + | "name" : "jung" + |} + """.stripMargin + + private val dummyResponseJson = + """ + |{ + | "action" : "DummyResponse", + | "serverSparkVersion" : "3.3.4", + | "success": true + |} + """.stripMargin + + private val submitDriverRequestJson = + """ + |{ + | "action" : "CreateSubmissionRequest", + | "appArgs" : [ "two slices", "a hint of cinnamon" ], + | "appResource" : "honey-walnut-cherry.jar", + | "clientSparkVersion" : "1.2.3", + | "environmentVariables" : { + | "PATH" : "/dev/null" + | }, + | "mainClass" : "org.apache.spark.examples.SparkPie", + | "sparkProperties" : { + | "spark.driver.extraLibraryPath" : "pickle.jar", + | "spark.jars" : "mayonnaise.jar,ketchup.jar", + | "spark.driver.supervise" : "false", + | "spark.app.name" : "SparkPie", + | "spark.cores.max" : "10000", + | "spark.driver.memory" : "512m", + | "spark.files" : "fireball.png", + | "spark.driver.cores" : "180", + | "spark.driver.extraJavaOptions" : " -Dslices=5 -Dcolor=mostly_red", + | "spark.executor.memory" : "256m", + | "spark.driver.extraClassPath" : "food-coloring.jar" + | } + |} + """.stripMargin + + private val submitDriverResponseJson = + """ + |{ + | "action" : "CreateSubmissionResponse", + | "serverSparkVersion" : "1.2.3", + | "submissionId" : "driver_123", + | "success" : true + |} + """.stripMargin + + private val killDriverResponseJson = + """ + |{ + | "action" : "KillSubmissionResponse", + | "serverSparkVersion" : "1.2.3", + | "submissionId" : "driver_123", + | "success" : true + |} + """.stripMargin + + private val driverStatusResponseJson = + """ + |{ + | "action" : "SubmissionStatusResponse", + | "driverState" : "RUNNING", + | "serverSparkVersion" : "1.2.3", + | "submissionId" : "driver_123", + | "success" : true, + | "workerHostPort" : "1.2.3.4:7780", + | "workerId" : "worker_123" + |} + """.stripMargin + + private val errorJson = + """ + |{ + | "action" : "ErrorResponse", + | "message" : "Field not found in submit request: X", + | "serverSparkVersion" : "1.2.3" + |} + """.stripMargin + + /** Assert that the contents in the two JSON strings are equal after ignoring whitespace. */ + private def assertJsonEquals(jsonString1: String, jsonString2: String): Unit = { + val trimmedJson1 = jsonString1.trim + val trimmedJson2 = jsonString2.trim + val json1 = compact(render(parse(trimmedJson1))) + val json2 = compact(render(parse(trimmedJson2))) + // Put this on a separate line to avoid printing comparison twice when test fails + val equals = json1 == json2 + assert(equals, "\"[%s]\" did not equal \"[%s]\"".format(trimmedJson1, trimmedJson2)) + } +} + +private class DummyResponse extends SubmitRestProtocolResponse +private class DummyRequest extends SubmitRestProtocolRequest { + var active: Boolean = null + var age: Integer = null + var name: String = null + protected override def doValidate(): Unit = { + super.doValidate() + assertFieldIsSet(name, "name") + assertFieldIsSet(age, "age") + assert(age > 5, "Not old enough!") + } +} diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala index 855f1b6276089..054a4c64897a9 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -29,9 +29,9 @@ class KryoSerializerDistributedSuite extends FunSuite { test("kryo objects are serialised consistently in different processes") { val conf = new SparkConf(false) - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryo.registrator", classOf[AppJarRegistrator].getName) - conf.set("spark.task.maxFailures", "1") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", classOf[AppJarRegistrator].getName) + .set("spark.task.maxFailures", "1") val jar = TestUtils.createJarWithClasses(List(AppJarRegistrator.customClassName)) conf.setJars(List(jar.getPath)) diff --git a/pom.xml b/pom.xml index aef450ae63121..da8ee077ddf85 100644 --- a/pom.xml +++ b/pom.xml @@ -154,6 +154,7 @@ org.scala-lang 3.6.3 1.8.8 + 2.4.4 1.1.1.6

      +### 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 170/578] [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 171/578] 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 172/578] 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 173/578] [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 174/578] [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 175/578] [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 176/578] [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 177/578] [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 178/578] [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 179/578] [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 180/578] [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 181/578] [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 182/578] [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 183/578] [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 184/578] [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 185/578] [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 186/578] [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 187/578] [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 188/578] 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 189/578] 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 190/578] [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("") {
    - + } } 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. + + + + + @@ -297,13 +306,11 @@ Apart from these, the following properties are also available, and may be useful - + @@ -865,8 +872,8 @@ Apart from these, the following properties are also available, and may be useful @@ -911,8 +918,8 @@ Apart from these, the following properties are also available, and may be useful @@ -920,7 +927,7 @@ Apart from these, the following properties are also available, and may be useful @@ -930,7 +937,7 @@ Apart from these, the following properties are also available, and may be useful @@ -939,7 +946,7 @@ Apart from these, the following properties are also available, and may be useful
    Executor IDTotal Tasks Failed Tasks Succeeded TasksInputOutputShuffle ReadShuffle WriteShuffle Spill (Memory)Shuffle Spill (Disk) + Input Size / Records + + Output Size / Records + + + Shuffle Read Size / Records + + + Shuffle Write Size / Records + Shuffle Spill (Memory)Shuffle Spill (Disk)
    {v.failedTasks + v.succeededTasks} {v.failedTasks} {v.succeededTasks} - {Utils.bytesToString(v.inputBytes)} - {Utils.bytesToString(v.outputBytes)} - {Utils.bytesToString(v.shuffleRead)} - {Utils.bytesToString(v.shuffleWrite)} - {Utils.bytesToString(v.memoryBytesSpilled)} - {Utils.bytesToString(v.diskBytesSpilled)} + {s"${Utils.bytesToString(v.inputBytes)} / ${v.inputRecords}"} + + {s"${Utils.bytesToString(v.outputBytes)} / ${v.outputRecords}"} + + {s"${Utils.bytesToString(v.shuffleRead)} / ${v.shuffleReadRecords}"} + + {s"${Utils.bytesToString(v.shuffleWrite)} / ${v.shuffleWriteRecords}"} + + {Utils.bytesToString(v.memoryBytesSpilled)} + + {Utils.bytesToString(v.diskBytesSpilled)} +
    {UIUtils.formatDuration(millis.toLong)}{Utils.bytesToString(d.toLong)}{Utils.bytesToString(d.toLong)}{s"${Utils.bytesToString(d.toLong)} / ${recordDist.next().toLong}"}InputInput Size / RecordsOutputOutput Size / RecordsShuffle Read (Remote)Shuffle Read Size / Records (Remote)Shuffle WriteShuffle Write Size / Records
    - {inputReadable} + {s"$inputReadable / $inputRecords"} - {outputReadable} + {s"$outputReadable / $outputRecords"} - {shuffleReadReadable} + {s"$shuffleReadReadable / $shuffleReadRecords"} - {shuffleWriteReadable} + {s"$shuffleWriteReadable / $shuffleWriteRecords"} {Utils.megabytesToString(driver.desc.mem.toLong)} {driver.desc.command.arguments(1)}{driver.desc.command.arguments(2)}
    spark.driver.userClassPathFirstfalse + (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)
    spark.files.userClassPathFirstspark.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.
    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. 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.
    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. 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.
    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 191/578] [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 192/578] [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 193/578] [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 194/578] [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 195/578] [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 196/578] 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 197/578] [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 198/578] [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 199/578] [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 200/578] [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 201/578] 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 202/578] [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 203/578] [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 204/578] [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 205/578] [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 206/578] [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 207/578] [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 208/578] [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 209/578] [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 210/578] [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 211/578] [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 212/578] [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 213/578] [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 214/578] [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 215/578] [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 216/578] [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 217/578] [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 218/578] [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 219/578] [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 220/578] [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 221/578] [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 222/578] [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 223/578] [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 224/578] 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 225/578] 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 226/578] [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 227/578] 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 228/578] 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 229/578] [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 230/578] [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 231/578] 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 232/578] [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 233/578] [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 234/578] [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 235/578] [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 236/578] [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 237/578] 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 238/578] 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 239/578] [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 240/578] [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 241/578] 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 242/578] [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 243/578] [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 244/578] [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 245/578] [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 246/578] [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 247/578] [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 248/578] [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 249/578] [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 250/578] 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 251/578] [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 252/578] [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 253/578] [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 254/578] [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 255/578] [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 256/578] [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 257/578] [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 258/578] [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 259/578] [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 260/578] [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 261/578] [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 262/578] [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 263/578] [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 264/578] [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 265/578] 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 266/578] [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 267/578] [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 268/578] [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 269/578] [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 270/578] [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 271/578] [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 272/578] [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 273/578] [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 274/578] 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 275/578] [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 276/578] [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 277/578] 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 278/578] 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 279/578] [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 280/578] [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 281/578] 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 282/578] [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 283/578] 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 284/578] [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 285/578] [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 286/578] [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 287/578] [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 288/578] [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 289/578] 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 290/578] [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 291/578] 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 292/578] 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 293/578] [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 294/578] [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 295/578] [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 296/578] [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 297/578] [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 298/578] [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 299/578] 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 300/578] 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 301/578] [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 302/578] 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 303/578] [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 304/578] [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 305/578] [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 306/578] [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 307/578] 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 308/578] [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 309/578] [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 310/578] [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 311/578] [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 312/578] [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 313/578] 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 314/578] [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 315/578] [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 316/578] 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 317/578] [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 318/578] 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 319/578] 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 320/578] [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 321/578] [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 322/578] [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 323/578] [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 324/578] [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 325/578] [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 326/578] 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 327/578] 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 328/578] [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 329/578] [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 330/578] [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 331/578] [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 332/578] [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 333/578] [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 334/578] [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 335/578] [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 336/578] [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 337/578] [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 338/578] [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 339/578] [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 340/578] [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 341/578] [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 342/578] [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 343/578] [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 344/578] [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 345/578] [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 346/578] 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 347/578] [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 348/578] 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 349/578] [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 350/578] 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 351/578] [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 352/578] [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 353/578] [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 354/578] [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 355/578] [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 356/578] [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 357/578] 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 358/578] [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 359/578] [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 360/578] [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 361/578] [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 362/578] [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 363/578] [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 364/578] 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 365/578] 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 366/578] [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 367/578] [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 368/578] [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 369/578] 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 370/578] [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 371/578] [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 372/578] [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 373/578] 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 374/578] [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 375/578] [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 376/578] 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 377/578] =?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 378/578] [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 379/578] [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 380/578] [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 381/578] [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 382/578] [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 383/578] [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 384/578] [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 385/578] [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 386/578] [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 387/578] [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 388/578] [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 389/578] [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 390/578] [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 391/578] [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 392/578] [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 393/578] [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 394/578] [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 395/578] [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 396/578] [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 397/578] [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 398/578] [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 399/578] [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 400/578] [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 401/578] [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 402/578] [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 403/578] [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 404/578] [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 405/578] [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 406/578] [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 407/578] [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 408/578] [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 409/578] [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 410/578] 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 411/578] [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 412/578] [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 413/578] [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 414/578] [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 415/578] [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 416/578] [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 417/578] [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 418/578] [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 419/578] [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 420/578] [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 421/578] [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 422/578] [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 423/578] [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 424/578] [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 425/578] [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 426/578] [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 427/578] [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 428/578] 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 429/578] 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 430/578] [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 431/578] [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 432/578] [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 433/578] 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 434/578] 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 435/578] [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 436/578] 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 437/578] 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 438/578] [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 439/578] [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 440/578] [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 441/578] [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 442/578] [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 443/578] 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 444/578] [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 445/578] 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 446/578] [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 447/578] [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 448/578] [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 449/578] [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 450/578] [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 451/578] [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 452/578] [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 453/578] [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 454/578] [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 455/578] 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 456/578] [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 457/578] 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 458/578] 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 459/578] 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 460/578] 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 461/578] [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 462/578] [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 463/578] [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 464/578] [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 465/578] [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 466/578] 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 467/578] [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 468/578] [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 469/578] [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 470/578] [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 471/578] [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 472/578] 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 473/578] [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 474/578] [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 475/578] 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 476/578] [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 477/578] [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 478/578] [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 479/578] [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 480/578] [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 481/578] [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 482/578] [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 483/578] [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 484/578] [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 485/578] [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 486/578] [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 487/578] [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 488/578] [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 489/578] [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 490/578] [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 491/578] [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 492/578] [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 493/578] 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 494/578] 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 495/578] [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 496/578] [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 497/578] 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 498/578] 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 499/578] [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 500/578] [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 501/578] [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 502/578] [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 503/578] [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 504/578] 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 505/578] [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 506/578] [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 507/578] [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 508/578] [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 509/578] [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 510/578] [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 511/578] 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 512/578] [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 513/578] 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 514/578] [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 515/578] [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 516/578] [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 517/578] [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 518/578] [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 519/578] [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 520/578] [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 521/578] [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 522/578] [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 523/578] 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 524/578] [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 525/578] [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 526/578] [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 527/578] [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 528/578] 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 529/578] [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 530/578] [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 531/578] [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 532/578] [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 533/578] [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 534/578] [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 535/578] [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 536/578] [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 537/578] [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 538/578] 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 539/578] [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 540/578] [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 541/578] [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 542/578] [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 543/578] [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 544/578] [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 545/578] 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 546/578] [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 547/578] [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 548/578] 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 549/578] 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 550/578] [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 551/578] [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 552/578] 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 553/578] [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 554/578] [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 555/578] [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 565/578] [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 566/578] [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 567/578] [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 568/578] [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 569/578] [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 570/578] [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 571/578] [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 572/578] [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 573/578] [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 574/578] [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 575/578] [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 576/578] [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 577/578] [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 578/578] [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", """