From 5820961289eb98e45eb467efa316c7592b8d619c Mon Sep 17 00:00:00 2001 From: Brennon York Date: Tue, 10 Feb 2015 14:57:00 -0800 Subject: [PATCH 001/152] [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 002/152] [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 003/152] [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 004/152] [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 005/152] [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 006/152] [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 007/152] [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 008/152] [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 009/152] [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 010/152] [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 011/152] [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 012/152] [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 013/152] [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 014/152] [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 015/152] [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 016/152] [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 017/152] 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 018/152] 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 019/152] [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 020/152] 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 021/152] 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 022/152] [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 023/152] [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 024/152] 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 025/152] [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 026/152] [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 027/152] [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 028/152] [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 029/152] [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 030/152] 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 031/152] 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 032/152] [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 033/152] [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 034/152] 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 035/152] [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 036/152] [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 037/152] [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 038/152] [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 039/152] [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 040/152] [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 041/152] [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 042/152] [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 043/152] 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 044/152] [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 045/152] [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 046/152] [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 047/152] [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 048/152] [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 049/152] [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 050/152] [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 051/152] [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 052/152] [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 053/152] [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 054/152] [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 055/152] [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 056/152] [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 057/152] [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 058/152] 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 059/152] [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 060/152] [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 061/152] [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 062/152] [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 063/152] [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 064/152] [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 065/152] [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 066/152] [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 067/152] 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 068/152] [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 069/152] [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 070/152] 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 071/152] 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 072/152] [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 073/152] [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 074/152] 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 075/152] [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 076/152] 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 077/152] [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 078/152] [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 079/152] [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 080/152] [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 081/152] [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 082/152] 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 083/152] [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 084/152] 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 085/152] 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 086/152] [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 087/152] [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 088/152] [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 089/152] [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 090/152] [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 091/152] [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 092/152] 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 093/152] 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 094/152] [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 095/152] 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 096/152] [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 097/152] [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 098/152] [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 099/152] [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 100/152] 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 101/152] [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 102/152] [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 103/152] [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 104/152] [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 105/152] [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 106/152] 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 107/152] [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 108/152] [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 109/152] 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 110/152] [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 111/152] 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 112/152] 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 113/152] [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 114/152] [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 115/152] [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 116/152] [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 117/152] [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 118/152] [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 119/152] 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 120/152] 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 121/152] [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 122/152] [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 123/152] [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 124/152] [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 125/152] [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 126/152] [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 127/152] [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 128/152] [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 129/152] [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 130/152] [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 131/152] [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 132/152] [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 133/152] [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 134/152] [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 135/152] [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 136/152] [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 137/152] [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 138/152] [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 139/152] 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 140/152] [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 141/152] 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 142/152] [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 143/152] 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 144/152] [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 145/152] [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 146/152] [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 147/152] [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 148/152] [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 149/152] [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 150/152] 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 151/152] [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 152/152] [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() + } } }